소스 검색

HADOOP-6240. Add new FileContext rename operation that posix compliant that allows overwriting existing destination. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@816794 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 16 년 전
부모
커밋
1695ecd1a3

+ 3 - 0
CHANGES.txt

@@ -203,6 +203,9 @@ Trunk (unreleased changes)
     HADOOP-6267. Permit building contrib modules located in external
     source trees.  (Todd Lipcon via cutting)
 
+    HADOOP-6240. Add new FileContext rename operation that posix compliant
+    that allows overwriting existing destination. (suresh)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

+ 30 - 105
src/java/org/apache/hadoop/fs/FileContext.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Progressable;
@@ -454,98 +456,6 @@ public final class FileContext {
     return path.makeQualified(defaultFS.getUri(), getWorkingDirectory());
   } 
 
-  /**
-   * Class to support the varargs for create() options.
-   *
-   */
-  public static class CreateOpts {
-    private CreateOpts() { };
-    public static BlockSize blockSize(long bs) { 
-      return new BlockSize(bs);
-    }
-    public static BufferSize bufferSize(short bs) { 
-      return new BufferSize(bs);
-    }
-    public static ReplicationFactor repFac(short rf) { 
-      return new ReplicationFactor(rf);
-    }
-    public static BytesPerChecksum bytesPerChecksum(short crc) {
-      return new BytesPerChecksum(crc);
-    }
-    public static Perms perms(FsPermission perm) {
-      return new Perms(perm);
-    }
-    
-    static class BlockSize extends CreateOpts {
-      private final long blockSize;
-      protected BlockSize(long bs) {
-        if (bs <= 0) {
-          throw new IllegalArgumentException(
-                        "Block size must be greater than 0");
-        }
-        blockSize = bs; 
-      }
-      long getValue() { return blockSize; }
-    }
-    
-    static class ReplicationFactor extends CreateOpts {
-      private final short replication;
-      protected ReplicationFactor(short rf) { 
-        if (rf <= 0) {
-          throw new IllegalArgumentException(
-                      "Replication must be greater than 0");
-        }
-        replication = rf;
-      }
-      short getValue() { return replication; }
-    }
-    
-    static class BufferSize extends CreateOpts {
-      private final int bufferSize;
-      protected BufferSize(short bs) {
-        if (bs <= 0) {
-          throw new IllegalArgumentException(
-                        "Buffer size must be greater than 0");
-        }
-        bufferSize = bs; 
-      }
-      int getValue() { return bufferSize; }
-    }
-    
-    static class BytesPerChecksum extends CreateOpts {
-      private final int bytesPerChecksum;
-      protected BytesPerChecksum(short bpc) { 
-        if (bpc <= 0) {
-          throw new IllegalArgumentException(
-                        "Bytes per checksum must be greater than 0");
-        }
-        bytesPerChecksum = bpc; 
-      }
-      int getValue() { return bytesPerChecksum; }
-    }
-    
-    static class Perms extends CreateOpts {
-      private final FsPermission permissions;
-      protected Perms(FsPermission perm) { 
-        if(perm == null) {
-          throw new IllegalArgumentException("Permissions must not be null");
-        }
-        permissions = perm; 
-      }
-      FsPermission getValue() { return permissions; }
-    }
-    
-    static class Progress extends CreateOpts {
-      private final Progressable progress;
-      protected Progress(Progressable prog) { 
-        if(prog == null) {
-          throw new IllegalArgumentException("Progress must not be null");
-        }
-        progress = prog;
-      }
-      Progressable getValue() { return progress; }
-    }
-  }
   
   /**
    * Create or overwrite file on indicated path and returns an output stream
@@ -717,15 +627,33 @@ public final class FileContext {
     return getFSofPath(absF).setReplication(absF, replication);
   }
 
- /**
-  * Renames Path src to Path dst. 
-  *  
-  * @param src
-  * @param dst
-  * @throws IOException if a rename is attempted across URI filessystem or
-  * across volumes within a file system.
-  */
-  public void rename(final Path src, final Path dst)
+  /**
+   * Renames Path src to Path dst
+   * <ul>
+   * <li
+   * <li>Fails if src is a file and dst is a directory.
+   * <li>Fails if src is a directory and dst is a file.
+   * <li>Fails if the parent of dst does not exist or is a file.
+   * </ul>
+   * <p>
+   * If OVERWRITE option is not passed as an argument, rename fails
+   * if the dst already exists.
+   * <p>
+   * If OVERWRITE option is passed as an argument, rename overwrites
+   * the dst if it is a file or an empty directory. Rename fails if dst is
+   * a non-empty directory.
+   * <p>
+   * Note that atomicity of rename is dependent on the file system
+   * implementation. Please refer to the file system documentation for
+   * details
+   * <p>
+   * 
+   * @param src path to be renamed
+   * @param dst new path after rename
+   * @throws IOException on failure
+   */
+  @SuppressWarnings("deprecation")
+  public void rename(final Path src, final Path dst, final Rename... options)
     throws IOException {
     final Path absSrc  = fixRelativePart(src);
     final Path absDst = fixRelativePart(dst);
@@ -734,10 +662,7 @@ public final class FileContext {
     if(!srcFS.getUri().equals(dstFS.getUri())) {
       throw new IOException("Renames across FileSystems not supported");
     }
-    if(srcFS.rename(absSrc, absDst)) {
-      return;
-    }
-    throw new IOException("bug in underlying filesystem");
+    srcFS.rename(absSrc, absDst, options);
   }
   
   /**

+ 91 - 1
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -41,6 +41,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -689,9 +690,98 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Renames Path src to Path dst.  Can take place on local fs
    * or remote DFS.
+   * @throws IOException on failure
+   * @return true if rename is successful
    */
   public abstract boolean rename(Path src, Path dst) throws IOException;
-    
+
+  /**
+   * Renames Path src to Path dst
+   * <ul>
+   * <li
+   * <li>Fails if src is a file and dst is a directory.
+   * <li>Fails if src is a directory and dst is a file.
+   * <li>Fails if the parent of dst does not exist or is a file.
+   * </ul>
+   * <p>
+   * If OVERWRITE option is not passed as an argument, rename fails
+   * if the dst already exists.
+   * <p>
+   * If OVERWRITE option is passed as an argument, rename overwrites
+   * the dst if it is a file or an empty directory. Rename fails if dst is
+   * a non-empty directory.
+   * <p>
+   * Note that atomicity of rename is dependent on the file system
+   * implementation. Please refer to the file system documentation for
+   * details. This default implementation is non atomic.
+   * <p>
+   * This method is deprecated since it is a temporary method added to 
+   * support the transition from FileSystem to FileContext for user 
+   * applications.
+   * 
+   * @param src path to be renamed
+   * @param dst new path after rename
+   * @throws IOException on failure
+   */
+  @Deprecated
+  protected void rename(final Path src, final Path dst,
+      final Rename... options) throws IOException {
+    // Default implementation
+    final FileStatus srcStatus = getFileStatus(src);
+    if (srcStatus == null) {
+      throw new FileNotFoundException("rename source " + src + " not found.");
+    }
+
+    boolean overwrite = false;
+    if (null != options) {
+      for (Rename option : options) {
+        if (option == Rename.OVERWRITE) {
+          overwrite = true;
+        }
+      }
+    }
+
+    FileStatus dstStatus;
+    try {
+      dstStatus = getFileStatus(dst);
+    } catch (IOException e) {
+      dstStatus = null;
+    }
+    if (dstStatus != null) {
+      if (srcStatus.isDir() != dstStatus.isDir()) {
+        throw new IOException("Source " + src + " Destination " + dst
+            + " both should be either file or directory");
+      }
+      if (!overwrite) {
+        throw new FileAlreadyExistsException("rename destination " + dst
+            + " already exists.");
+      }
+      // Delete the destination that is a file or an empty directory
+      if (dstStatus.isDir()) {
+        FileStatus[] list = listStatus(dst);
+        if (list != null && list.length != 0) {
+          throw new IOException(
+              "rename cannot overwrite non empty destination directory " + dst);
+        }
+      }
+      delete(dst, false);
+    } else {
+      final Path parent = dst.getParent();
+      final FileStatus parentStatus = getFileStatus(parent);
+      if (parentStatus == null) {
+        throw new FileNotFoundException("rename destination parent " + parent
+            + " not found.");
+      }
+      if (!parentStatus.isDir()) {
+        throw new ParentNotDirectoryException("rename destination parent " + parent
+            + " is a file.");
+      }
+    }
+    if (!rename(src, dst)) {
+      throw new IOException("rename from " + src + " to " + dst + " failed.");
+    }
+  }
+  
   /** Delete a file.
    *
    * @param f the path to delete.

+ 141 - 0
src/java/org/apache/hadoop/fs/Options.java

@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * This class contains options related to file system operations.
+ */
+public class Options {
+  /**
+   * Class to support the varargs for create() options.
+   *
+   */
+  public static class CreateOpts {
+    private CreateOpts() { };
+    public static BlockSize blockSize(long bs) { 
+      return new BlockSize(bs);
+    }
+    public static BufferSize bufferSize(short bs) { 
+      return new BufferSize(bs);
+    }
+    public static ReplicationFactor repFac(short rf) { 
+      return new ReplicationFactor(rf);
+    }
+    public static BytesPerChecksum bytesPerChecksum(short crc) {
+      return new BytesPerChecksum(crc);
+    }
+    public static Perms perms(FsPermission perm) {
+      return new Perms(perm);
+    }
+    
+    static class BlockSize extends CreateOpts {
+      private final long blockSize;
+      protected BlockSize(long bs) {
+        if (bs <= 0) {
+          throw new IllegalArgumentException(
+                        "Block size must be greater than 0");
+        }
+        blockSize = bs; 
+      }
+      long getValue() { return blockSize; }
+    }
+    
+    static class ReplicationFactor extends CreateOpts {
+      private final short replication;
+      protected ReplicationFactor(short rf) { 
+        if (rf <= 0) {
+          throw new IllegalArgumentException(
+                      "Replication must be greater than 0");
+        }
+        replication = rf;
+      }
+      short getValue() { return replication; }
+    }
+    
+    static class BufferSize extends CreateOpts {
+      private final int bufferSize;
+      protected BufferSize(short bs) {
+        if (bs <= 0) {
+          throw new IllegalArgumentException(
+                        "Buffer size must be greater than 0");
+        }
+        bufferSize = bs; 
+      }
+      int getValue() { return bufferSize; }
+    }
+    
+    static class BytesPerChecksum extends CreateOpts {
+      private final int bytesPerChecksum;
+      protected BytesPerChecksum(short bpc) { 
+        if (bpc <= 0) {
+          throw new IllegalArgumentException(
+                        "Bytes per checksum must be greater than 0");
+        }
+        bytesPerChecksum = bpc; 
+      }
+      int getValue() { return bytesPerChecksum; }
+    }
+    
+    static class Perms extends CreateOpts {
+      private final FsPermission permissions;
+      protected Perms(FsPermission perm) { 
+        if(perm == null) {
+          throw new IllegalArgumentException("Permissions must not be null");
+        }
+        permissions = perm; 
+      }
+      FsPermission getValue() { return permissions; }
+    }
+    
+    static class Progress extends CreateOpts {
+      private final Progressable progress;
+      protected Progress(Progressable prog) { 
+        if(prog == null) {
+          throw new IllegalArgumentException("Progress must not be null");
+        }
+        progress = prog;
+      }
+      Progressable getValue() { return progress; }
+    }
+  }
+
+  /**
+   * Enum to support the varargs for rename() options
+   */
+  public static enum Rename {
+    NONE((byte) 0), // No options
+    OVERWRITE((byte) 1); // Overwrite the rename destination
+
+    private final byte code;
+    
+    private Rename(byte code) {
+      this.code = code;
+    }
+
+    public static Rename valueOf(byte code) {
+      return code < 0 || code >= values().length ? null : values()[code];
+    }
+
+    public byte value() {
+      return code;
+    }
+  }
+}

+ 36 - 0
src/java/org/apache/hadoop/fs/ParentNotDirectoryException.java

@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+/**
+ * Indicates that the parent of specified Path is not a directory
+ * as expected.
+ */
+public class ParentNotDirectoryException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public ParentNotDirectoryException() {
+    super();
+  }
+
+  public ParentNotDirectoryException(String msg) {
+    super(msg);
+  }
+}

+ 166 - 77
src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -20,20 +20,15 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.EnumSet;
-import java.util.Random;
 
-import org.junit.*;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileContext.CreateOpts;
+import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * <p>
@@ -96,6 +91,11 @@ public abstract class FileContextMainOperationsBaseTest  {
     return true;
   }
 
+  
+  protected IOException unwrapException(IOException e) {
+    return e;
+  }
+  
   @Test
   public void testFsStatus() throws Exception {
     FsStatus fsStatus = fc.getFsStatus(null);
@@ -402,36 +402,77 @@ public abstract class FileContextMainOperationsBaseTest  {
   @Test
   public void testRenameNonExistentPath() throws Exception {
     if (!renameSupported()) return;
-    Path src = getTestRootPath("test/hadoop/NonExistingPath");
+    Path src = getTestRootPath("test/hadoop/nonExistent");
     Path dst = getTestRootPath("test/new/newpath");
     try {
-      fc.rename(src, dst);
-      Assert.assertTrue("rename of non existing path should have Assert.failed",
-                                                    false);
-    } catch (Exception e) {
-      // expected
+      rename(src, dst, false, false, false, Rename.NONE);
+      Assert.fail("Should throw FileNotFoundException");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
+    }
+
+    try {
+      rename(src, dst, false, false, false, Rename.OVERWRITE);
+      Assert.fail("Should throw FileNotFoundException");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
     }
   }
 
   @Test
-  public void testRenameFileMoveToNonExistentDirectory() throws Exception {
+  public void testRenameFileToNonExistentDirectory() throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/file");
     createFile(src);
-    Path dst = getTestRootPath("test/NonExisting/foo");
-    rename(src, dst, false, true, false);
+    Path dst = getTestRootPath("test/nonExistent/newfile");
+    
+    try {
+      rename(src, dst, false, true, false, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
+    }
+
+    try {
+      rename(src, dst, false, true, false, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
+    }
   }
 
   @Test
-  public void testRenameFileMoveToExistingDirectory() throws Exception {
+  public void testRenameFileToDestinationWithParentFile() throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/file");
     createFile(src);
-    Path dst = getTestRootPath("test/Existing/newfile");
-    fc.mkdirs(dst.getParent(), FsPermission.getDefault());
-    rename(src, dst, true, false, true);
+    Path dst = getTestRootPath("test/parentFile/newfile");
+    createFile(dst.getParent());
+    
+    try {
+      rename(src, dst, false, true, false, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+    }
+
+    try {
+      rename(src, dst, false, true, false, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+    }
+  }
+
+  @Test
+  public void testRenameFileToExistingParent() throws Exception {
+    if (!renameSupported()) return;
+    
+    Path src = getTestRootPath("test/hadoop/file");
+    createFile(src);
+    Path dst = getTestRootPath("test/new/newfile");
+    fc.mkdirs(dst.getParent(), FileContext.DEFAULT_PERM);
+    rename(src, dst, true, false, true, Rename.OVERWRITE);
   }
 
   @Test
@@ -440,9 +481,19 @@ public abstract class FileContextMainOperationsBaseTest  {
     
     Path src = getTestRootPath("test/hadoop/file");
     createFile(src);
-    Path dst = getTestRootPath("test/existing/existingFile");
+    Path dst = getTestRootPath("test/new/existingFile");
     createFile(dst);
-    rename(src, dst, true, false, true);
+    
+    // Fails without overwrite option
+    try {
+      rename(src, dst, false, true, false, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileAlreadyExistsException);
+    }
+    
+    // Succeeds with overwrite option
+    rename(src, dst, true, false, true, Rename.OVERWRITE);
   }
 
   @Test
@@ -451,37 +502,66 @@ public abstract class FileContextMainOperationsBaseTest  {
     
     Path src = getTestRootPath("test/hadoop/file");
     createFile(src);
-    Path dst = getTestRootPath("test/existing/existingDir");
-    fc.mkdirs(dst, FsPermission.getDefault());
-    rename(src, dst, true, false, true);
-    Assert.assertTrue("Destination changed",
-        fc.exists(getTestRootPath("test/existing/existingDir/file")));
+    Path dst = getTestRootPath("test/new/existingDir");
+    fc.mkdirs(dst, FileContext.DEFAULT_PERM);
+    
+    // Fails without overwrite option
+    try {
+      rename(src, dst, false, false, true, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+    }
+    
+    // File cannot be renamed as directory
+    try {
+      rename(src, dst, false, false, true, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+    }
   }
-  
+
   @Test
-  public void testRenameDirectoryMoveToNonExistentDirectory() 
-    throws Exception {
+  public void testRenameDirectoryToNonExistentParent() throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/dir");
-    fc.mkdirs(src, FsPermission.getDefault());
-    Path dst = getTestRootPath("test/nonExisting/newdir");
-    rename(src, dst, false, true, false);
+    fc.mkdirs(src, FileContext.DEFAULT_PERM);
+    Path dst = getTestRootPath("test/nonExistent/newdir");
+    
+    try {
+      rename(src, dst, false, true, false, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
+    }
+
+    try {
+      rename(src, dst, false, true, false, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileNotFoundException);
+    }
   }
-  
+
   @Test
-  public void testRenameDirectoryMoveToExistingDirectory() throws Exception {
+  public void testRenameDirectoryAsNonExistentDirectory() throws Exception {
+    testRenameDirectoryAsNonExistentDirectory(Rename.NONE);
+    tearDown();
+    testRenameDirectoryAsNonExistentDirectory(Rename.OVERWRITE);
+  }
+
+  private void testRenameDirectoryAsNonExistentDirectory(Rename... options) throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/dir");
-    fc.mkdirs(src, FsPermission.getDefault());
+    fc.mkdirs(src, FileContext.DEFAULT_PERM);
     createFile(getTestRootPath("test/hadoop/dir/file1"));
     createFile(getTestRootPath("test/hadoop/dir/subdir/file2"));
     
     Path dst = getTestRootPath("test/new/newdir");
-    fc.mkdirs(dst.getParent(), FsPermission.getDefault());
-    rename(src, dst, true, false, true);
+    fc.mkdirs(dst.getParent(), FileContext.DEFAULT_PERM);
     
+    rename(src, dst, true, false, true, options);
     Assert.assertFalse("Nested file1 exists",
         fc.exists(getTestRootPath("test/hadoop/dir/file1")));
     Assert.assertFalse("Nested file2 exists",
@@ -491,40 +571,55 @@ public abstract class FileContextMainOperationsBaseTest  {
     Assert.assertTrue("Renamed nested exists",
         fc.exists(getTestRootPath("test/new/newdir/subdir/file2")));
   }
-  
+
   @Test
-  public void testRenameDirectoryAsExistingFile() throws Exception {
+  public void testRenameDirectoryAsNonEmptyDirectory() throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/dir");
-    fc.mkdirs(src, FsPermission.getDefault());
-    Path dst = getTestRootPath("test/new/newfile");
-    createFile(dst);
-    rename(src, dst, false, true, true);
+    fc.mkdirs(src, FileContext.DEFAULT_PERM);
+    createFile(getTestRootPath("test/hadoop/dir/file1"));
+    createFile(getTestRootPath("test/hadoop/dir/subdir/file2"));
+    
+    Path dst = getTestRootPath("test/new/newdir");
+    fc.mkdirs(dst, FileContext.DEFAULT_PERM);
+    createFile(getTestRootPath("test/new/newdir/file1"));
+    // Fails without overwrite option
+    try {
+      rename(src, dst, false, true, false, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+      Assert.assertTrue(unwrapException(e) instanceof FileAlreadyExistsException);
+    }
+    // Succeeds with overwrite option
+    try {
+      rename(src, dst, false, true, false, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException ex) {
+      // Expected exception
+    }
   }
-  
+
   @Test
-  public void testRenameDirectoryAsExistingDirectory() throws Exception {
+  public void testRenameDirectoryAsFile() throws Exception {
     if (!renameSupported()) return;
     
     Path src = getTestRootPath("test/hadoop/dir");
-    fc.mkdirs(src, FsPermission.getDefault());
-    createFile(getTestRootPath("test/hadoop/dir/file1"));
-    createFile(getTestRootPath("test/hadoop/dir/subdir/file2"));
-    
-    Path dst = getTestRootPath("test/new/newdir");
-    fc.mkdirs(dst, FsPermission.getDefault());
-    rename(src, dst, true, false, true);
-    Assert.assertTrue("Destination changed",
-        fc.exists(getTestRootPath("test/new/newdir/dir")));    
-    Assert.assertFalse("Nested file1 exists",
-        fc.exists(getTestRootPath("test/hadoop/dir/file1")));
-    Assert.assertFalse("Nested file2 exists",
-        fc.exists(getTestRootPath("test/hadoop/dir/subdir/file2")));
-    Assert.assertTrue("Renamed nested file1 exists",
-        fc.exists(getTestRootPath("test/new/newdir/dir/file1")));
-    Assert.assertTrue("Renamed nested exists",
-        fc.exists(getTestRootPath("test/new/newdir/dir/subdir/file2")));
+    fc.mkdirs(src, FileContext.DEFAULT_PERM);
+    Path dst = getTestRootPath("test/new/newfile");
+    createFile(dst);
+    // Fails without overwrite option
+    try {
+      rename(src, dst, false, true, true, Rename.NONE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException e) {
+    }
+    // Directory cannot be renamed as existing file
+    try {
+      rename(src, dst, false, true, true, Rename.OVERWRITE);
+      Assert.fail("Expected exception is not thrown");
+    } catch (IOException ex) {
+    }
   }
 
   @Test
@@ -556,16 +651,10 @@ public abstract class FileContextMainOperationsBaseTest  {
   }
   
   private void rename(Path src, Path dst, boolean renameShouldSucceed,
-      boolean srcExists, boolean dstExists) throws IOException {
-    try {
-      fc.rename(src, dst);
-      if (!renameShouldSucceed)
-        Assert.fail("rename should have thrown exception");
-    } catch (Exception e) {
-      if (renameShouldSucceed)
-        Assert.fail("rename should have suceeded, but threw exception");
-    }
-
+      boolean srcExists, boolean dstExists, Rename... options) throws IOException {
+    fc.rename(src, dst, options);
+    if (!renameShouldSucceed)
+      Assert.fail("rename should have thrown exception");
     Assert.assertEquals("Source exists", srcExists, fc.exists(src));
     Assert.assertEquals("Destination exists", dstExists, fc.exists(dst));
   }

+ 0 - 19
src/test/core/org/apache/hadoop/fs/TestLocalFSFileContextMainOperations.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Before;
-import org.junit.Test;
 
 public class TestLocalFSFileContextMainOperations extends FileContextMainOperationsBaseTest {
 
@@ -38,22 +37,4 @@ public class TestLocalFSFileContextMainOperations extends FileContextMainOperati
       wd = FileSystem.getLocal(new Configuration()).getWorkingDirectory();
     return wd;
   }
-  
-  @Override
-  @Test
-  public void testRenameFileMoveToNonExistentDirectory() throws Exception {
-    // ignore base class test till hadoop-6240 is fixed
-  }
-    
-  @Override
-  @Test
-  public void testRenameDirectoryMoveToNonExistentDirectory() 
-    throws Exception {
-    // ignore base class test till hadoop-6240 is fixed
-  }
-  @Override
-  @Test
-  public void testRenameDirectoryAsExistingDirectory() throws Exception {
-    // ignore base class test till hadoop-6240 is fixed
-  }
 }