HADOOP-9805. Refactor RawLocalFileSystem#rename for improved testability. Contributed by Jean-Pierre Matsumoto.
(cherry picked from commit 5763b173d3
)
This commit is contained in:
parent
1278310c21
commit
aabd8c8e04
|
@ -39,6 +39,9 @@ Release 2.8.0 - UNRELEASED
|
||||||
HADOOP-11660. Add support for hardware crc of HDFS checksums on ARM aarch64
|
HADOOP-11660. Add support for hardware crc of HDFS checksums on ARM aarch64
|
||||||
architecture (Edward Nevill via Colin P. McCabe)
|
architecture (Edward Nevill via Colin P. McCabe)
|
||||||
|
|
||||||
|
HADOOP-9805. Refactor RawLocalFileSystem#rename for improved testability.
|
||||||
|
(Jean-Pierre Matsumoto via cnauroth)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.permission.FsPermission;
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||||
import org.apache.hadoop.io.nativeio.NativeIOException;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.apache.hadoop.util.Shell;
|
import org.apache.hadoop.util.Shell;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
@ -347,11 +346,29 @@ public class RawLocalFileSystem extends FileSystem {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Enforce POSIX rename behavior that a source directory replaces an existing
|
// Else try POSIX style rename on Windows only
|
||||||
// destination if the destination is an empty directory. On most platforms,
|
if (Shell.WINDOWS &&
|
||||||
// this is already handled by the Java API call above. Some platforms
|
handleEmptyDstDirectoryOnWindows(src, srcFile, dst, dstFile)) {
|
||||||
// (notably Windows) do not provide this behavior, so the Java API call above
|
return true;
|
||||||
// fails. Delete destination and attempt rename again.
|
}
|
||||||
|
|
||||||
|
// The fallback behavior accomplishes the rename by a full copy.
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Falling through to a copy of " + src + " to " + dst);
|
||||||
|
}
|
||||||
|
return FileUtil.copy(this, src, this, dst, true, getConf());
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public final boolean handleEmptyDstDirectoryOnWindows(Path src, File srcFile,
|
||||||
|
Path dst, File dstFile) throws IOException {
|
||||||
|
|
||||||
|
// Enforce POSIX rename behavior that a source directory replaces an
|
||||||
|
// existing destination if the destination is an empty directory. On most
|
||||||
|
// platforms, this is already handled by the Java API call above. Some
|
||||||
|
// platforms (notably Windows) do not provide this behavior, so the Java API
|
||||||
|
// call renameTo(dstFile) fails. Delete destination and attempt rename
|
||||||
|
// again.
|
||||||
if (this.exists(dst)) {
|
if (this.exists(dst)) {
|
||||||
FileStatus sdst = this.getFileStatus(dst);
|
FileStatus sdst = this.getFileStatus(dst);
|
||||||
if (sdst.isDirectory() && dstFile.list().length == 0) {
|
if (sdst.isDirectory() && dstFile.list().length == 0) {
|
||||||
|
@ -364,12 +381,7 @@ public class RawLocalFileSystem extends FileSystem {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return false;
|
||||||
// The fallback behavior accomplishes the rename by a full copy.
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("Falling through to a copy of " + src + " to " + dst);
|
|
||||||
}
|
|
||||||
return FileUtil.copy(this, src, this, dst, true, getConf());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,8 +19,13 @@
|
||||||
package org.apache.hadoop.fs.contract.rawlocal;
|
package org.apache.hadoop.fs.contract.rawlocal;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.RawLocalFileSystem;
|
||||||
import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
|
import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
|
||||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||||
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestRawlocalContractRename extends AbstractContractRenameTest {
|
public class TestRawlocalContractRename extends AbstractContractRenameTest {
|
||||||
|
|
||||||
|
@ -29,4 +34,46 @@ public class TestRawlocalContractRename extends AbstractContractRenameTest {
|
||||||
return new RawlocalFSContract(conf);
|
return new RawlocalFSContract(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test fallback rename code <code>handleEmptyDstDirectoryOnWindows()</code>
|
||||||
|
* even on not Windows platform where the normal <code>File.renameTo()</code>
|
||||||
|
* is supposed to work well. This test has been added for HADOOP-9805.
|
||||||
|
*
|
||||||
|
* @see AbstractContractRenameTest#testRenameWithNonEmptySubDirPOSIX()
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testRenameWithNonEmptySubDirPOSIX() throws Throwable {
|
||||||
|
final Path renameTestDir = path("testRenameWithNonEmptySubDir");
|
||||||
|
final Path srcDir = new Path(renameTestDir, "src1");
|
||||||
|
final Path srcSubDir = new Path(srcDir, "sub");
|
||||||
|
final Path finalDir = new Path(renameTestDir, "dest");
|
||||||
|
FileSystem fs = getFileSystem();
|
||||||
|
ContractTestUtils.rm(fs, renameTestDir, true, false);
|
||||||
|
|
||||||
|
fs.mkdirs(srcDir);
|
||||||
|
fs.mkdirs(finalDir);
|
||||||
|
ContractTestUtils.writeTextFile(fs, new Path(srcDir, "source.txt"),
|
||||||
|
"this is the file in src dir", false);
|
||||||
|
ContractTestUtils.writeTextFile(fs, new Path(srcSubDir, "subfile.txt"),
|
||||||
|
"this is the file in src/sub dir", false);
|
||||||
|
|
||||||
|
ContractTestUtils.assertPathExists(fs, "not created in src dir",
|
||||||
|
new Path(srcDir, "source.txt"));
|
||||||
|
ContractTestUtils.assertPathExists(fs, "not created in src/sub dir",
|
||||||
|
new Path(srcSubDir, "subfile.txt"));
|
||||||
|
|
||||||
|
RawLocalFileSystem rlfs = (RawLocalFileSystem) fs;
|
||||||
|
rlfs.handleEmptyDstDirectoryOnWindows(srcDir, rlfs.pathToFile(srcDir),
|
||||||
|
finalDir, rlfs.pathToFile(finalDir));
|
||||||
|
|
||||||
|
// Accept only POSIX rename behavior in this test
|
||||||
|
ContractTestUtils.assertPathExists(fs, "not renamed into dest dir",
|
||||||
|
new Path(finalDir, "source.txt"));
|
||||||
|
ContractTestUtils.assertPathExists(fs, "not renamed into dest/sub dir",
|
||||||
|
new Path(finalDir, "sub/subfile.txt"));
|
||||||
|
|
||||||
|
ContractTestUtils.assertPathDoesNotExist(fs, "not deleted",
|
||||||
|
new Path(srcDir, "source.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue