|
@@ -18,22 +18,28 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.DFSClientAdapter;
|
|
|
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.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
@@ -182,5 +188,97 @@ public class TestINodeFileUnderConstructionWithSnapshot {
|
|
|
|
|
|
// re-check the size of nodeInDeleted_S1
|
|
|
assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(s1));
|
|
|
- }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * call DFSClient#callGetBlockLocations(...) for snapshot file. Make sure only
|
|
|
+ * blocks within the size range are returned.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testGetBlockLocations() throws Exception {
|
|
|
+ final Path root = new Path("/");
|
|
|
+ final Path file = new Path("/file");
|
|
|
+ DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
|
|
|
+
|
|
|
+ // take a snapshot on root
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
|
|
|
+
|
|
|
+ final Path fileInSnapshot = SnapshotTestHelper.getSnapshotPath(root,
|
|
|
+ "s1", file.getName());
|
|
|
+ FileStatus status = hdfs.getFileStatus(fileInSnapshot);
|
|
|
+ // make sure we record the size for the file
|
|
|
+ assertEquals(BLOCKSIZE, status.getLen());
|
|
|
+
|
|
|
+ // append data to file
|
|
|
+ DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE - 1);
|
|
|
+ status = hdfs.getFileStatus(fileInSnapshot);
|
|
|
+ // the size of snapshot file should still be BLOCKSIZE
|
|
|
+ assertEquals(BLOCKSIZE, status.getLen());
|
|
|
+ // the size of the file should be (2 * BLOCKSIZE - 1)
|
|
|
+ status = hdfs.getFileStatus(file);
|
|
|
+ assertEquals(BLOCKSIZE * 2 - 1, status.getLen());
|
|
|
+
|
|
|
+ // call DFSClient#callGetBlockLocations for the file in snapshot
|
|
|
+ LocatedBlocks blocks = DFSClientAdapter.callGetBlockLocations(
|
|
|
+ cluster.getNameNodeRpc(), fileInSnapshot.toString(), 0, Long.MAX_VALUE);
|
|
|
+ List<LocatedBlock> blockList = blocks.getLocatedBlocks();
|
|
|
+
|
|
|
+ // should be only one block
|
|
|
+ assertEquals(BLOCKSIZE, blocks.getFileLength());
|
|
|
+ assertEquals(1, blockList.size());
|
|
|
+
|
|
|
+ // check the last block
|
|
|
+ LocatedBlock lastBlock = blocks.getLastLocatedBlock();
|
|
|
+ assertEquals(0, lastBlock.getStartOffset());
|
|
|
+ assertEquals(BLOCKSIZE, lastBlock.getBlockSize());
|
|
|
+
|
|
|
+ // take another snapshot
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, root, "s2");
|
|
|
+ final Path fileInSnapshot2 = SnapshotTestHelper.getSnapshotPath(root,
|
|
|
+ "s2", file.getName());
|
|
|
+
|
|
|
+ // append data to file without closing
|
|
|
+ HdfsDataOutputStream out = appendFileWithoutClosing(file, BLOCKSIZE);
|
|
|
+ out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
|
|
+
|
|
|
+ status = hdfs.getFileStatus(fileInSnapshot2);
|
|
|
+ // the size of snapshot file should be BLOCKSIZE*2-1
|
|
|
+ assertEquals(BLOCKSIZE * 2 - 1, status.getLen());
|
|
|
+ // the size of the file should be (3 * BLOCKSIZE - 1)
|
|
|
+ status = hdfs.getFileStatus(file);
|
|
|
+ assertEquals(BLOCKSIZE * 3 - 1, status.getLen());
|
|
|
+
|
|
|
+ blocks = DFSClientAdapter.callGetBlockLocations(cluster.getNameNodeRpc(),
|
|
|
+ fileInSnapshot2.toString(), 0, Long.MAX_VALUE);
|
|
|
+ assertFalse(blocks.isUnderConstruction());
|
|
|
+ assertTrue(blocks.isLastBlockComplete());
|
|
|
+ blockList = blocks.getLocatedBlocks();
|
|
|
+
|
|
|
+ // should be 2 blocks
|
|
|
+ assertEquals(BLOCKSIZE * 2 - 1, blocks.getFileLength());
|
|
|
+ assertEquals(2, blockList.size());
|
|
|
+
|
|
|
+ // check the last block
|
|
|
+ lastBlock = blocks.getLastLocatedBlock();
|
|
|
+ assertEquals(BLOCKSIZE, lastBlock.getStartOffset());
|
|
|
+ assertEquals(BLOCKSIZE, lastBlock.getBlockSize());
|
|
|
+
|
|
|
+ blocks = DFSClientAdapter.callGetBlockLocations(cluster.getNameNodeRpc(),
|
|
|
+ fileInSnapshot2.toString(), BLOCKSIZE, 0);
|
|
|
+ blockList = blocks.getLocatedBlocks();
|
|
|
+ assertEquals(1, blockList.size());
|
|
|
+
|
|
|
+ // check blocks for file being written
|
|
|
+ blocks = DFSClientAdapter.callGetBlockLocations(cluster.getNameNodeRpc(),
|
|
|
+ file.toString(), 0, Long.MAX_VALUE);
|
|
|
+ blockList = blocks.getLocatedBlocks();
|
|
|
+ assertEquals(3, blockList.size());
|
|
|
+ assertTrue(blocks.isUnderConstruction());
|
|
|
+ assertFalse(blocks.isLastBlockComplete());
|
|
|
+
|
|
|
+ lastBlock = blocks.getLastLocatedBlock();
|
|
|
+ assertEquals(BLOCKSIZE * 2, lastBlock.getStartOffset());
|
|
|
+ assertEquals(BLOCKSIZE - 1, lastBlock.getBlockSize());
|
|
|
+ out.close();
|
|
|
+ }
|
|
|
}
|