HDFS-5425. Merge change r1541261 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1541271 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jing Zhao 2013-11-12 22:15:04 +00:00
parent 9b3b133edf
commit f1aa8a866c
5 changed files with 70 additions and 4 deletions

View File

@ -160,6 +160,9 @@ Release 2.3.0 - UNRELEASED
HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on
restart. (jing9 and Vinay)
Release 2.2.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -822,7 +822,12 @@ public class FSImageFormat {
final INodesInPath iip = fsDir.getLastINodeInPath(path);
INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
cons.setLocalName(oldnode.getLocalNameBytes());
cons.setParent(oldnode.getParent());
INodeReference parentRef = oldnode.getParentReference();
if (parentRef != null) {
cons.setParentReference(parentRef);
} else {
cons.setParent(oldnode.getParent());
}
if (oldnode instanceof INodeFileWithSnapshot) {
cons = new INodeFileUnderConstructionWithSnapshot(cons,

View File

@ -2524,7 +2524,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
final INode[] inodes = analyzeFileState(
src, fileId, clientName, previous, onRetryBlock).getINodes();
final INodeFileUnderConstruction pendingFile =
(INodeFileUnderConstruction) inodes[inodes.length - 1];
(INodeFileUnderConstruction) inodes[inodes.length - 1].asFile();
if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
// This is a retry. Just return the last block if having locations.
@ -2562,7 +2562,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
final INode[] inodes = inodesInPath.getINodes();
final INodeFileUnderConstruction pendingFile =
(INodeFileUnderConstruction) inodes[inodes.length - 1];
(INodeFileUnderConstruction) inodes[inodes.length - 1].asFile();
if (onRetryBlock[0] != null) {
if (onRetryBlock[0].getLocations().length > 0) {

View File

@ -595,7 +595,15 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
public void replaceChild(final INode oldChild, final INode newChild,
final INodeMap inodeMap) {
super.replaceChild(oldChild, newChild, inodeMap);
diffs.replaceChild(ListType.CREATED, oldChild, newChild);
if (oldChild.getParentReference() != null && !newChild.isReference()) {
// oldChild is referred by a Reference node. Thus we are replacing the
// referred inode, e.g.,
// INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
// in this case, we do not need to update the diff list
return;
} else {
diffs.replaceChild(ListType.CREATED, oldChild, newChild);
}
}
/**

View File

@ -29,6 +29,7 @@ import static org.mockito.Mockito.spy;
import java.io.File;
import java.io.IOException;
import java.util.EnumSet;
import java.util.List;
import java.util.Random;
@ -40,9 +41,12 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@ -102,6 +106,7 @@ public class TestRenameWithSnapshots {
@Before
public void setUp() throws Exception {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).format(true)
.build();
cluster.waitActive();
@ -2289,4 +2294,49 @@ public class TestRenameWithSnapshots {
assertEquals(0, diff.getChildrenDiff().getList(ListType.DELETED).size());
assertEquals(0, diff.getChildrenDiff().getList(ListType.CREATED).size());
}
/**
* Rename of the underconstruction file in snapshot should not fail NN restart
* after checkpoint. Unit test for HDFS-5425.
*/
@Test
public void testRenameUCFileInSnapshot() throws Exception {
final Path test = new Path("/test");
final Path foo = new Path(test, "foo");
final Path bar = new Path(foo, "bar");
hdfs.mkdirs(foo);
// create a file and keep it as underconstruction.
hdfs.create(bar);
SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
// rename bar --> bar2
final Path bar2 = new Path(foo, "bar2");
hdfs.rename(bar, bar2);
// save namespace and restart
restartClusterAndCheckImage(true);
}
/**
* Similar with testRenameUCFileInSnapshot, but do renaming first and then
* append file without closing it. Unit test for HDFS-5425.
*/
@Test
public void testAppendFileAfterRenameInSnapshot() throws Exception {
final Path test = new Path("/test");
final Path foo = new Path(test, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
// rename bar --> bar2
final Path bar2 = new Path(foo, "bar2");
hdfs.rename(bar, bar2);
// append file and keep it as underconstruction.
FSDataOutputStream out = hdfs.append(bar2);
out.writeByte(0);
((DFSOutputStream) out.getWrappedStream()).hsync(
EnumSet.of(SyncFlag.UPDATE_LENGTH));
// save namespace and restart
restartClusterAndCheckImage(true);
}
}