diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index 0b09f02b93982..3573792598491 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -477,11 +477,11 @@ running out of memory as it calculates the partitions.
Any FileSystem that does not actually break files into blocks SHOULD
return a number for this that results in efficient processing.
-A FileSystem MAY make this user-configurable (the S3 and Swift filesystem clients do this).
+A FileSystem MAY make this user-configurable (the object store connectors usually do this).
### `long getDefaultBlockSize(Path p)`
-Get the "default" block size for a path —that is, the block size to be used
+Get the "default" block size for a path --that is, the block size to be used
when writing objects to a path in the filesystem.
#### Preconditions
@@ -530,14 +530,21 @@ on the filesystem.
### `boolean mkdirs(Path p, FsPermission permission)`
-Create a directory and all its parents
+Create a directory and all its parents.
#### Preconditions
+The path must either be a directory or not exist
+
if exists(FS, p) and not isDir(FS, p) :
raise [ParentNotDirectoryException, FileAlreadyExistsException, IOException]
+No ancestor may be a file
+
+ forall d = ancestors(FS, p) :
+ if exists(FS, d) and not isDir(FS, d) :
+ raise [ParentNotDirectoryException, FileAlreadyExistsException, IOException]
#### Postconditions
@@ -577,6 +584,11 @@ Writing to or overwriting a directory must fail.
if isDir(FS, p) : raise {FileAlreadyExistsException, FileNotFoundException, IOException}
+No ancestor may be a file
+
+ forall d = ancestors(FS, p) :
+ if exists(FS, d) and not isDir(FS, d) :
+ raise [ParentNotDirectoryException, FileAlreadyExistsException, IOException]
FileSystems may reject the request for other
reasons, such as the FS being read-only (HDFS),
@@ -584,7 +596,8 @@ the block size being below the minimum permitted (HDFS),
the replication count being out of range (HDFS),
quotas on namespace or filesystem being exceeded, reserved
names, etc. All rejections SHOULD be `IOException` or a subclass thereof
-and MAY be a `RuntimeException` or subclass. For instance, HDFS may raise a `InvalidPathException`.
+and MAY be a `RuntimeException` or subclass.
+For instance, HDFS may raise an `InvalidPathException`.
#### Postconditions
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java
index 07c99e0b6a528..79222ce67d6cf 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java
@@ -22,11 +22,11 @@
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.junit.Test;
-import org.junit.internal.AssumptionViolatedException;
+import org.junit.AssumptionViolatedException;
-import java.io.FileNotFoundException;
import java.io.IOException;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
@@ -40,7 +40,7 @@
* Test creating files, overwrite options etc.
*/
public abstract class AbstractContractCreateTest extends
- AbstractFSContractTestBase {
+ AbstractFSContractTestBase {
/**
* How long to wait for a path to become visible.
@@ -113,7 +113,6 @@ private void testOverwriteExistingFile(boolean useBuilder) throws Throwable {
* This test catches some eventual consistency problems that blobstores exhibit,
* as we are implicitly verifying that updates are consistent. This
* is why different file lengths and datasets are used
- * @throws Throwable
*/
@Test
public void testOverwriteExistingFile() throws Throwable {
@@ -137,10 +136,6 @@ private void testOverwriteEmptyDirectory(boolean useBuilder)
} catch (FileAlreadyExistsException expected) {
//expected
handleExpectedException(expected);
- } catch (FileNotFoundException e) {
- handleRelaxedException("overwriting a dir with a file ",
- "FileAlreadyExistsException",
- e);
} catch (IOException e) {
handleRelaxedException("overwriting a dir with a file ",
"FileAlreadyExistsException",
@@ -189,10 +184,6 @@ private void testOverwriteNonEmptyDirectory(boolean useBuilder)
} catch (FileAlreadyExistsException expected) {
//expected
handleExpectedException(expected);
- } catch (FileNotFoundException e) {
- handleRelaxedException("overwriting a dir with a file ",
- "FileAlreadyExistsException",
- e);
} catch (IOException e) {
handleRelaxedException("overwriting a dir with a file ",
"FileAlreadyExistsException",
@@ -332,4 +323,117 @@ public void testCreateMakesParentDirs() throws Throwable {
assertTrue("Grandparent directory does not appear to be a directory",
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);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
index 2751294beb92c..5312a88f1aef8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
@@ -29,10 +29,10 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
/**
- * Test creating files, overwrite options &c
+ * Test renaming files.
*/
public abstract class AbstractContractRenameTest extends
- AbstractFSContractTestBase {
+ AbstractFSContractTestBase {
@Test
public void testRenameNewFileSameDir() throws Throwable {
@@ -83,7 +83,8 @@ public void testRenameNonexistentFile() throws Throwable {
"FileNotFoundException",
e);
}
- assertPathDoesNotExist("rename nonexistent file created a destination file", target);
+ assertPathDoesNotExist("rename nonexistent file created a destination file",
+ target);
}
/**
@@ -112,7 +113,7 @@ public void testRenameFileOverExistingFile() throws Throwable {
// the filesystem supports rename(file, file2) by overwriting file2
assertTrue("Rename returned false", renamed);
- destUnchanged = false;
+ destUnchanged = false;
} else {
// rename is rejected by returning 'false' or throwing an exception
if (renamed && !renameReturnsFalseOnRenameDestExists) {
@@ -129,12 +130,13 @@ public void testRenameFileOverExistingFile() throws Throwable {
// verify that the destination file is as expected based on the expected
// outcome
verifyFileContents(getFileSystem(), destFile,
- destUnchanged? destData: srcData);
+ destUnchanged ? destData: srcData);
}
@Test
public void testRenameDirIntoExistingDir() throws Throwable {
- describe("Verify renaming a dir into an existing dir puts it underneath"
+ describe("Verify renaming a dir into an existing dir puts it"
+ + " underneath"
+" and leaves existing files alone");
FileSystem fs = getFileSystem();
String sourceSubdir = "source";
@@ -145,15 +147,15 @@ public void testRenameDirIntoExistingDir() throws Throwable {
Path destDir = path("dest");
Path destFilePath = new Path(destDir, "dest-512.txt");
- byte[] destDateset = dataset(512, 'A', 'Z');
- writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024, false);
+ byte[] destData = dataset(512, 'A', 'Z');
+ writeDataset(fs, destFilePath, destData, destData.length, 1024, false);
assertIsFile(destFilePath);
boolean rename = rename(srcDir, destDir);
Path renamedSrc = new Path(destDir, sourceSubdir);
assertIsFile(destFilePath);
assertIsDirectory(renamedSrc);
- verifyFileContents(fs, destFilePath, destDateset);
+ verifyFileContents(fs, destFilePath, destData);
assertTrue("rename returned false though the contents were copied", rename);
}
@@ -285,4 +287,54 @@ protected void validateAncestorsMoved(Path src, Path dst, String nestedPath)
}
}
+ @Test
+ public void testRenameFileUnderFile() throws Exception {
+ String action = "rename directly under file";
+ describe(action);
+ Path base = methodPath();
+ Path grandparent = new Path(base, "file");
+ expectRenameUnderFileFails(action,
+ grandparent,
+ new Path(base, "testRenameSrc"),
+ new Path(grandparent, "testRenameTarget"));
+ }
+
+ @Test
+ public void testRenameFileUnderFileSubdir() throws Exception {
+ String action = "rename directly under file/subdir";
+ describe(action);
+ Path base = methodPath();
+ Path grandparent = new Path(base, "file");
+ Path parent = new Path(grandparent, "parent");
+ expectRenameUnderFileFails(action,
+ grandparent,
+ new Path(base, "testRenameSrc"),
+ new Path(parent, "testRenameTarget"));
+ }
+
+ protected void expectRenameUnderFileFails(String action,
+ Path file, Path renameSrc, Path renameTarget)
+ throws Exception {
+ byte[] data = dataset(256, 'a', 'z');
+ FileSystem fs = getFileSystem();
+ writeDataset(fs, file, data, data.length, 1024 * 1024,
+ true);
+ writeDataset(fs, renameSrc, data, data.length, 1024 * 1024,
+ true);
+ String outcome;
+ boolean renamed;
+ try {
+ renamed = rename(renameSrc, renameTarget);
+ outcome = action + ": rename (" + renameSrc + ", " + renameTarget
+ + ")= " + renamed;
+ } catch (IOException e) {
+ // raw local raises an exception here
+ renamed = false;
+ outcome = "rename raised an exception: " + e;
+ }
+ assertPathDoesNotExist("after " + outcome, renameTarget);
+ assertFalse(outcome, renamed);
+ assertPathExists(action, renameSrc);
+ }
+
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
index 1cd2164fad300..217c3aeb7742a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
@@ -225,6 +225,15 @@ protected Path path(String filepath) throws IOException {
new Path(getContract().getTestPath(), filepath));
}
+ /**
+ * Get a path whose name ends with the name of this method.
+ * @return a path implicitly unique amongst all methods in this class
+ * @throws IOException IO problems
+ */
+ protected Path methodPath() throws IOException {
+ return path(methodName.getMethodName());
+ }
+
/**
* Take a simple path like "/something" and turn it into
* a qualified path against the test FS.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
index 91a112141e987..3f31c07742c59 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
@@ -51,6 +51,15 @@ public interface ContractOptions {
*/
String CREATE_VISIBILITY_DELAYED = "create-visibility-delayed";
+ /**
+ * Flag to indicate that it is possible to create a file under a file.
+ * This is a complete violation of the filesystem rules, but it is one
+ * which object stores have been known to do for performance
+ * and because nobody has ever noticed.
+ * {@value}
+ */
+ String CREATE_FILE_UNDER_FILE_ALLOWED = "create-file-under-file-allowed";
+
/**
* Is a filesystem case sensitive.
* Some of the filesystems that say "no" here may mean
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 56280f3a8bcff..077bf5230ead6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1543,7 +1543,8 @@ public boolean rename(String src, String dst) throws IOException {
DSQuotaExceededException.class,
QuotaByStorageTypeExceededException.class,
UnresolvedPathException.class,
- SnapshotAccessControlException.class);
+ SnapshotAccessControlException.class,
+ ParentNotDirectoryException.class);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/contract/s3a.xml b/hadoop-tools/hadoop-aws/src/test/resources/contract/s3a.xml
index 0e12897266091..f6b0e406b3bb2 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/contract/s3a.xml
+++ b/hadoop-tools/hadoop-aws/src/test/resources/contract/s3a.xml
@@ -127,4 +127,9 @@
true
+
+ fs.contract.create-file-under-file-allowed
+ true
+
+
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
index d72d35e92ee0a..3e3a010e17484 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
@@ -19,9 +19,13 @@
package org.apache.hadoop.fs.adl.live;
+import org.junit.Test;
+
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.test.LambdaTestUtils;
/**
* Test rename contract test cases on Adl file system.
@@ -32,4 +36,15 @@ public class TestAdlContractRenameLive extends AbstractContractRenameTest {
protected AbstractFSContract createContract(Configuration configuration) {
return new AdlStorageContract(configuration);
}
+
+ /**
+ * ADL throws an Access Control Exception rather than return false.
+ * This is caught and its error text checked, to catch regressions.
+ */
+ @Test
+ public void testRenameFileUnderFile() throws Exception {
+ LambdaTestUtils.intercept(AccessControlException.class,
+ "Parent path is not a folder.",
+ super::testRenameFileUnderFile);
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java
index 7820b7e65d51e..aab2607b8f809 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java
@@ -38,7 +38,7 @@ private static int getFilesCreated(AzureBlobStorageTestAccount testAccount) {
/**
* Tests that when we have multiple file systems created/destroyed
* metrics from each are published correctly.
- * @throws Exception
+ * @throws Exception on a failure
*/
@Test
public void testMetricsAcrossFileSystems()
@@ -46,26 +46,37 @@ public void testMetricsAcrossFileSystems()
AzureBlobStorageTestAccount a1, a2, a3;
a1 = AzureBlobStorageTestAccount.createMock();
- assertEquals(0, getFilesCreated(a1));
+ assertFilesCreated(a1, "a1", 0);
a2 = AzureBlobStorageTestAccount.createMock();
- assertEquals(0, getFilesCreated(a2));
+ assertFilesCreated(a2, "a2", 0);
a1.getFileSystem().create(new Path("/foo")).close();
a1.getFileSystem().create(new Path("/bar")).close();
a2.getFileSystem().create(new Path("/baz")).close();
- assertEquals(0, getFilesCreated(a1));
- assertEquals(0, getFilesCreated(a2));
+ assertFilesCreated(a1, "a1", 0);
+ assertFilesCreated(a2, "a2", 0);
a1.closeFileSystem(); // Causes the file system to close, which publishes metrics
a2.closeFileSystem();
-
- assertEquals(2, getFilesCreated(a1));
- assertEquals(1, getFilesCreated(a2));
+
+ assertFilesCreated(a1, "a1", 2);
+ assertFilesCreated(a2, "a2", 1);
a3 = AzureBlobStorageTestAccount.createMock();
- assertEquals(0, getFilesCreated(a3));
+ assertFilesCreated(a3, "a3", 0);
a3.closeFileSystem();
- assertEquals(0, getFilesCreated(a3));
+ assertFilesCreated(a3, "a3", 0);
+ }
+
+ /**
+ * Assert that a specific number of files were created.
+ * @param account account to examine
+ * @param name account name (for exception text)
+ * @param expected expected value
+ */
+ private void assertFilesCreated(AzureBlobStorageTestAccount account,
+ String name, int expected) {
+ assertEquals("Files created in account " + name,
+ expected, getFilesCreated(account));
}
-
@Test
public void testMetricsSourceNames() {
String name1 = NativeAzureFileSystem.newMetricsSourceName();
@@ -83,6 +94,6 @@ public void testSkipMetricsCollection() throws Exception {
NativeAzureFileSystem.SKIP_AZURE_METRICS_PROPERTY_NAME, true);
a.getFileSystem().create(new Path("/foo")).close();
a.closeFileSystem(); // Causes the file system to close, which publishes metrics
- assertEquals(0, getFilesCreated(a));
+ assertFilesCreated(a, "a", 0);
}
}
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java
index 510cb984d8129..560eadd930917 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java
@@ -24,6 +24,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -46,6 +47,7 @@
import java.io.OutputStream;
import java.net.URI;
import java.util.ArrayList;
+import java.util.EnumSet;
import java.util.List;
/**
@@ -596,14 +598,12 @@ public boolean rename(Path src, Path dst) throws IOException {
store.rename(makeAbsolute(src), makeAbsolute(dst));
//success
return true;
- } catch (SwiftOperationFailedException e) {
- //downgrade to a failure
- return false;
- } catch (FileAlreadyExistsException e) {
- //downgrade to a failure
- return false;
- } catch (FileNotFoundException e) {
+ } catch (SwiftOperationFailedException
+ | FileAlreadyExistsException
+ | FileNotFoundException
+ | ParentNotDirectoryException e) {
//downgrade to a failure
+ LOG.debug("rename({}, {}) failed",src, dst, e);
return false;
}
}
@@ -733,4 +733,29 @@ public static long getBytesUploaded(FSDataOutputStream outputStream) {
return snos.getBytesUploaded();
}
+ /**
+ * {@inheritDoc}
+ * @throws FileNotFoundException if the parent directory is not present -or
+ * is not a directory.
+ */
+ @Override
+ public FSDataOutputStream createNonRecursive(Path path,
+ FsPermission permission,
+ EnumSet flags,
+ int bufferSize,
+ short replication,
+ long blockSize,
+ Progressable progress) throws IOException {
+ Path parent = path.getParent();
+ if (parent != null) {
+ // expect this to raise an exception if there is no parent
+ if (!getFileStatus(parent).isDirectory()) {
+ throw new FileAlreadyExistsException("Not a directory: " + parent);
+ }
+ }
+ return create(path, permission,
+ flags.contains(CreateFlag.OVERWRITE), bufferSize,
+ replication, blockSize, progress);
+ }
+
}
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
index ed7a782284bc3..5e4800900920a 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.swift.exceptions.SwiftConfigurationException;
import org.apache.hadoop.fs.swift.exceptions.SwiftException;
@@ -562,13 +563,17 @@ public void rename(Path src, Path dst)
//parent dir (in which case the dest dir exists), or the destination
//directory is root, in which case it must also exist
if (dstParent != null && !dstParent.equals(srcParent)) {
+ SwiftFileStatus fileStatus;
try {
- getObjectMetadata(dstParent);
+ fileStatus = getObjectMetadata(dstParent);
} catch (FileNotFoundException e) {
//destination parent doesn't exist; bail out
LOG.debug("destination parent directory " + dstParent + " doesn't exist");
throw e;
}
+ if (!fileStatus.isDir()) {
+ throw new ParentNotDirectoryException(dstParent.toString());
+ }
}
boolean destExists = dstMetadata != null;