Return-Path: X-Original-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id B513FE489 for ; Fri, 25 Jan 2013 03:09:55 +0000 (UTC) Received: (qmail 45831 invoked by uid 500); 25 Jan 2013 03:09:55 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 45611 invoked by uid 500); 25 Jan 2013 03:09:54 -0000 Mailing-List: contact hdfs-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-dev@hadoop.apache.org Delivered-To: mailing list hdfs-commits@hadoop.apache.org Received: (qmail 45582 invoked by uid 99); 25 Jan 2013 03:09:53 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jan 2013 03:09:53 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jan 2013 03:09:48 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 9025223888E7; Fri, 25 Jan 2013 03:09:27 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1438304 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/ src/test/java/org/apache/hadoop/hdfs/se... Date: Fri, 25 Jan 2013 03:09:27 -0000 To: hdfs-commits@hadoop.apache.org From: szetszwo@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130125030927.9025223888E7@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: szetszwo Date: Fri Jan 25 03:09:26 2013 New Revision: 1438304 URL: http://svn.apache.org/viewvc?rev=1438304&view=rev Log: HDFS-4429. When the latest snapshot exists, INodeFileUnderConstruction should be replaced with INodeFileWithSnapshot but not INodeFile. Added: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt Fri Jan 25 03:09:26 2013 @@ -120,3 +120,7 @@ Branch-2802 Snapshot (Unreleased) HDFS-4436. Change INode.recordModification(..) to return only the current inode and remove the updateCircularList parameter from some methods in INodeDirectoryWithSnapshot.Diff. (szetszwo) + + HDFS-4429. When the latest snapshot exists, INodeFileUnderConstruction should + be replaced with INodeFileWithSnapshot but not INodeFile. (Jing Zhao + via szetszwo) Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Jan 25 03:09:26 2013 @@ -178,6 +178,7 @@ import org.apache.hadoop.hdfs.server.nam import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable; +import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager; @@ -3293,6 +3294,21 @@ public class FSNamesystem implements Nam throws IOException, UnresolvedLinkException { assert hasWriteLock(); leaseManager.removeLease(pendingFile.getClientName(), src); + + if (latestSnapshot != null) { + if (!(pendingFile instanceof INodeFileUnderConstructionWithSnapshot)) { + // replace INodeFileUnderConstruction with + // INodeFileUnderConstructionWithSnapshot. This replacement does not + // need to be recorded in snapshot. + INodeFileUnderConstructionWithSnapshot pendingFileWithSnaphsot = + new INodeFileUnderConstructionWithSnapshot(pendingFile); + dir.replaceINodeFile(src, pendingFile, + pendingFileWithSnaphsot, null); + pendingFile = pendingFileWithSnaphsot; + } + pendingFile = (INodeFileUnderConstruction) pendingFile + .recordModification(latestSnapshot); + } // The file is no longer pending. // Create permanent INode, update blocks Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java Fri Jan 25 03:09:26 2013 @@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.nam import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot; +import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.util.ReadOnlyList; @@ -431,8 +432,8 @@ public abstract class INode implements C @VisibleForTesting public String getObjectString() { - final String s = super.toString(); - return s.substring(s.lastIndexOf(getClass().getSimpleName())); + return getClass().getSimpleName() + "@" + + Integer.toHexString(super.hashCode()); } @VisibleForTesting @@ -693,15 +694,21 @@ public abstract class INode implements C String blocksInfo = ((INodeFile) this).printBlocksInfo(); out.print(", blocks=[" + blocksInfo + "]"); } - if (this instanceof INodeFileWithSnapshot) { - INodeFileWithSnapshot nodeWithLink = (INodeFileWithSnapshot) this; - FileWithSnapshot next = nodeWithLink.getNext(); - out.print(", next=" - + (next != null ? next.asINodeFile().getObjectString() : "null")); - if (this instanceof INodeFileSnapshot) { + if (this instanceof FileWithSnapshot) { + if (this instanceof INodeFileSnapshot + || this instanceof INodeFileUnderConstructionSnapshot) { out.print(", computedSize=" + ((INodeFileSnapshot) this).computeFileSize(true)); } + FileWithSnapshot nodeWithLink = (FileWithSnapshot) this; + FileWithSnapshot next = nodeWithLink.getNext(); + // An INodeFileWithSnapshot whose next link pointing to itself should be + // equivalent with a normal INodeFile + if (!(this instanceof INodeFileWithSnapshot && + ((INodeFileWithSnapshot) this).getNext() == this)) { + out.print(", next=" + + (next != null ? next.asINodeFile().getObjectString() : "null")); + } } out.println(); } else { Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java Fri Jan 25 03:09:26 2013 @@ -528,7 +528,7 @@ public class INodeDirectoryWithSnapshot diff.deleted = deletedList; } - public Diff getDiff() { + Diff getDiff() { return diff; } Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java Fri Jan 25 03:09:26 2013 @@ -39,6 +39,20 @@ public class INodeFileUnderConstructionW super(f.asINodeFile(), clientName, clientMachine, clientNode); } + /** + * The constructor that creates an + * {@link INodeFileUnderConstructionWithSnapshot} based on an + * {@link INodeFileUnderConstruction} + * + * @param child The given {@link INodeFileUnderConstruction} instance + */ + public INodeFileUnderConstructionWithSnapshot( + INodeFileUnderConstruction child) { + super(child, child.getClientName(), child.getClientMachine(), child + .getClientNode()); + next = this; + } + @Override protected INodeFileWithSnapshot toINodeFile(final long mtime) { assertAllBlocksComplete(); @@ -46,7 +60,8 @@ public class INodeFileUnderConstructionW final INodeFileWithSnapshot f = new INodeFileWithSnapshot(this); f.setModificationTime(mtime, null); f.setAccessTime(atime, null); - Util.replace(this, f); + // link f with this + this.insertBefore(f); return f; } Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java Fri Jan 25 03:09:26 2013 @@ -43,7 +43,7 @@ public class INodeFileWithSnapshot exten final INodeFileUnderConstructionWithSnapshot f = new INodeFileUnderConstructionWithSnapshot(this, clientName, clientMachine, clientNode); - Util.replace(this, f); + this.insertBefore(f); return f; } Added: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java?rev=1438304&view=auto ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java (added) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java Fri Jan 25 03:09:26 2013 @@ -0,0 +1,240 @@ +/** + * 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.hdfs.server.namenode.snapshot; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +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; +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; +import org.apache.hadoop.hdfs.server.namenode.FSDirectory; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INode; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; +import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.Diff; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test snapshot functionalities while file appending. + */ +public class TestINodeFileUnderConstructionWithSnapshot { + + static final long seed = 0; + static final short REPLICATION = 3; + static final int BLOCKSIZE = 1024; + + private final Path dir = new Path("/TestSnapshot"); + + Configuration conf; + MiniDFSCluster cluster; + FSNamesystem fsn; + DistributedFileSystem hdfs; + FSDirectory fsdir; + + @Before + public void setUp() throws Exception { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION) + .build(); + cluster.waitActive(); + fsn = cluster.getNamesystem(); + fsdir = fsn.getFSDirectory(); + hdfs = cluster.getFileSystem(); + hdfs.mkdirs(dir); + } + + @After + public void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Check if the given nodes can form a circular list + */ + private void checkCircularList(FileWithSnapshot... nodes) { + for (int i = 0; i < nodes.length; i++) { + FileWithSnapshot next = nodes[i].getNext(); + FileWithSnapshot expectedNext = nodes[(i + 1) % nodes.length]; + Assert.assertTrue(next == expectedNext); + } + } + + /** + * Test snapshot after file appending + */ + @Test + public void testSnapshotAfterAppending() throws Exception { + Path file = new Path(dir, "file"); + // 1. create snapshot --> create file --> append + SnapshotTestHelper.createSnapshot(hdfs, dir, "s0"); + DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed); + DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE); + + // check the circular list and corresponding inodes: there should only be a + // reference of the current node in the created list + INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString()); + INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir + .getINode(dir.toString()); + Diff diff = dirNode.getLastSnapshotDiff().getDiff(); + INode nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes()); + assertTrue(fileNode == nodeInCreated); + INode nodeInDeleted = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertNull(nodeInDeleted); + + // 2. create snapshot --> modify the file --> append + hdfs.createSnapshot(dir, "s1"); + hdfs.setReplication(file, (short) (REPLICATION - 1)); + DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE); + + // check the circular list and corresponding inodes + diff = dirNode.getLastSnapshotDiff().getDiff(); + fileNode = (INodeFile) fsdir.getINode(file.toString()); + nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes()); + assertTrue(fileNode == nodeInCreated); + assertEquals(REPLICATION - 1, + ((INodeFile) nodeInCreated).getFileReplication()); + assertEquals(BLOCKSIZE * 3, ((INodeFile) fileNode).computeFileSize(true)); + nodeInDeleted = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertEquals(REPLICATION, + ((INodeFile) nodeInDeleted).getFileReplication()); + assertEquals(BLOCKSIZE * 2, + ((INodeFile) nodeInDeleted).computeFileSize(true)); + checkCircularList((INodeFileWithSnapshot) fileNode, + (INodeFileSnapshot) nodeInDeleted); + + // 3. create snapshot --> append + hdfs.createSnapshot(dir, "s2"); + DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE); + + // check the circular list and corresponding inodes + diff = dirNode.getLastSnapshotDiff().getDiff(); + fileNode = (INodeFile) fsdir.getINode(file.toString()); + nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes()); + assertTrue(fileNode == nodeInCreated); + assertEquals(REPLICATION - 1, + ((INodeFile) nodeInCreated).getFileReplication()); + assertEquals(BLOCKSIZE * 4, ((INodeFile) fileNode).computeFileSize(true)); + INode nodeInDeleted2 = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertEquals(REPLICATION - 1, + ((INodeFile) nodeInDeleted2).getFileReplication()); + assertEquals(BLOCKSIZE * 3, + ((INodeFile) nodeInDeleted2).computeFileSize(true)); + checkCircularList((INodeFileWithSnapshot) fileNode, + (INodeFileSnapshot) nodeInDeleted2, (INodeFileSnapshot) nodeInDeleted); + } + + private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length) + throws IOException { + byte[] toAppend = new byte[length]; + Random random = new Random(); + random.nextBytes(toAppend); + HdfsDataOutputStream out = (HdfsDataOutputStream) hdfs.append(file); + out.write(toAppend); + return out; + } + + /** + * Test snapshot during file appending, before the corresponding + * {@link FSDataOutputStream} instance closes. + */ + @Test + public void testSnapshotWhileAppending() throws Exception { + Path file = new Path(dir, "file"); + DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed); + + // 1. append without closing stream --> create snapshot + HdfsDataOutputStream out = appendFileWithoutClosing(file, BLOCKSIZE); + out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); + SnapshotTestHelper.createSnapshot(hdfs, dir, "s0"); + out.close(); + + // check: an INodeFileUnderConstructionSnapshot should be stored into s0's + // deleted list, with size BLOCKSIZE*2 + INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString()); + assertEquals(BLOCKSIZE * 2, ((INodeFile) fileNode).computeFileSize(true)); + INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir + .getINode(dir.toString()); + Diff diff = dirNode.getLastSnapshotDiff().getDiff(); + INode nodeInDeleted_S0 = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertTrue(nodeInDeleted_S0 instanceof INodeFileUnderConstructionSnapshot); + assertEquals(BLOCKSIZE * 2, + ((INodeFile) nodeInDeleted_S0).computeFileSize(true)); + + // 2. append without closing stream + out = appendFileWithoutClosing(file, BLOCKSIZE); + out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); + + // re-check nodeInDeleted_S0 + dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString()); + diff = dirNode.getLastSnapshotDiff().getDiff(); + nodeInDeleted_S0 = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertTrue(nodeInDeleted_S0 instanceof INodeFileUnderConstructionSnapshot); + assertEquals(BLOCKSIZE * 2, + ((INodeFile) nodeInDeleted_S0).computeFileSize(true)); + + // 3. take snapshot --> close stream + hdfs.createSnapshot(dir, "s1"); + out.close(); + + // check: an INodeFileUnderConstructionSnapshot with size BLOCKSIZE*3 should + // have been stored in s1's deleted list + fileNode = (INodeFile) fsdir.getINode(file.toString()); + dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString()); + diff = dirNode.getLastSnapshotDiff().getDiff(); + INode nodeInCreated_S1 = diff.searchCreated(fileNode.getLocalNameBytes()); + assertTrue(fileNode == nodeInCreated_S1); + assertTrue(fileNode instanceof INodeFileWithSnapshot); + INode nodeInDeleted_S1 = diff.searchDeleted(fileNode.getLocalNameBytes()); + assertTrue(nodeInDeleted_S1 instanceof INodeFileUnderConstructionSnapshot); + assertEquals(BLOCKSIZE * 3, + ((INodeFile) nodeInDeleted_S1).computeFileSize(true)); + // also check the circular linked list + checkCircularList((INodeFileWithSnapshot) fileNode, + (INodeFileUnderConstructionSnapshot) nodeInDeleted_S1, + (INodeFileUnderConstructionSnapshot) nodeInDeleted_S0); + + // 4. modify file --> append without closing stream --> take snapshot --> + // close stream + hdfs.setReplication(file, (short) (REPLICATION - 1)); + out = appendFileWithoutClosing(file, BLOCKSIZE); + hdfs.createSnapshot(dir, "s2"); + out.close(); + + // re-check the size of nodeInDeleted_S1 + assertEquals(BLOCKSIZE * 3, + ((INodeFile) nodeInDeleted_S1).computeFileSize(true)); + } +} \ No newline at end of file Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java?rev=1438304&r1=1438303&r2=1438304&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java (original) +++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java Fri Jan 25 03:09:26 2013 @@ -20,6 +20,12 @@ package org.apache.hadoop.hdfs.server.na import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -42,6 +48,7 @@ import org.apache.hadoop.hdfs.server.nam import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.util.Time; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -68,6 +75,9 @@ public class TestSnapshot { private static Random random = new Random(seed); + private static String testDir = + System.getProperty("test.build.data", "build/test/data"); + @Rule public ExpectedException exception = ExpectedException.none(); @@ -144,13 +154,23 @@ public class TestSnapshot { return nodes; } + private File getDumpTreeFile(String dir, String suffix) { + return new File(dir, String.format("dumptree_%s", suffix)); + } + /** * Restart the cluster to check edit log applying and fsimage saving/loading */ private void checkFSImage() throws Exception { + File fsnBefore = getDumpTreeFile(testDir, "before"); + File fsnMiddle = getDumpTreeFile(testDir, "middle"); + File fsnAfter = getDumpTreeFile(testDir, "after"); + String rootDir = "/"; - StringBuffer fsnStrBefore = fsn.getFSDirectory().getINode(rootDir) - .dumpTreeRecursively(); + PrintWriter out = new PrintWriter(new FileWriter(fsnBefore, false), true); + fsn.getFSDirectory().getINode(rootDir) + .dumpTreeRecursively(out, new StringBuilder(), null); + out.close(); cluster.shutdown(); cluster = new MiniDFSCluster.Builder(conf).format(false) @@ -158,11 +178,13 @@ public class TestSnapshot { cluster.waitActive(); fsn = cluster.getNamesystem(); hdfs = cluster.getFileSystem(); - // later check fsnStrMiddle to see if the edit log is recorded and applied + // later check fsnMiddle to see if the edit log is recorded and applied // correctly - StringBuffer fsnStrMiddle = fsn.getFSDirectory().getINode(rootDir) - .dumpTreeRecursively(); - + out = new PrintWriter(new FileWriter(fsnMiddle, false), true); + fsn.getFSDirectory().getINode(rootDir) + .dumpTreeRecursively(out, new StringBuilder(), null); + out.close(); + // save namespace and restart cluster hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER); hdfs.saveNamespace(); @@ -174,18 +196,38 @@ public class TestSnapshot { fsn = cluster.getNamesystem(); hdfs = cluster.getFileSystem(); // dump the namespace loaded from fsimage - StringBuffer fsnStrAfter = fsn.getFSDirectory().getINode(rootDir) - .dumpTreeRecursively(); - - System.out.println("================== Original FSDir =================="); - System.out.println(fsnStrBefore.toString()); - System.out.println("================== FSDir After Applying Edit Logs =================="); - System.out.println(fsnStrMiddle.toString()); - System.out.println("================ FSDir After FSImage Saving/Loading ================"); - System.out.println(fsnStrAfter.toString()); - System.out.println("===================================================="); - assertEquals(fsnStrBefore.toString(), fsnStrMiddle.toString()); - assertEquals(fsnStrBefore.toString(), fsnStrAfter.toString()); + out = new PrintWriter(new FileWriter(fsnAfter, false), true); + fsn.getFSDirectory().getINode(rootDir) + .dumpTreeRecursively(out, new StringBuilder(), null); + out.close(); + + compareFile(fsnBefore, fsnMiddle); + compareFile(fsnBefore, fsnAfter); + } + + /** compare two file's content */ + private void compareFile(File file1, File file2) throws IOException { + BufferedReader reader1 = new BufferedReader(new FileReader(file1)); + BufferedReader reader2 = new BufferedReader(new FileReader(file2)); + try { + String line1 = ""; + String line2 = ""; + while ((line1 = reader1.readLine()) != null + && (line2 = reader2.readLine()) != null) { + // skip the hashCode part of the object string during the comparison, + // also ignore the difference between INodeFile/INodeFileWithSnapshot + line1 = line1.replaceAll("INodeFileWithSnapshot", "INodeFile"); + line2 = line2.replaceAll("INodeFileWithSnapshot", "INodeFile"); + line1 = line1.replaceAll("@[\\dabcdef]+", ""); + line2 = line2.replaceAll("@[\\dabcdef]+", ""); + assertEquals(line1, line2); + } + Assert.assertNull(reader1.readLine()); + Assert.assertNull(reader2.readLine()); + } finally { + reader1.close(); + reader2.close(); + } } /**