|
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
|
@@ -274,4 +275,76 @@ public class TestSnapshotBlocksMap {
|
|
"s2/bar");
|
|
"s2/bar");
|
|
DFSTestUtil.readFile(hdfs, new Path(bar2SnapshotPath));
|
|
DFSTestUtil.readFile(hdfs, new Path(bar2SnapshotPath));
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Make sure we delete 0-sized block when deleting an INodeFileUCWithSnapshot
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testDeletionWithZeroSizeBlock() throws Exception {
|
|
|
|
+ final Path foo = new Path("/foo");
|
|
|
|
+ final Path bar = new Path(foo, "bar");
|
|
|
|
+ DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPLICATION, 0L);
|
|
|
|
+
|
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, foo, "s0");
|
|
|
|
+ hdfs.append(bar);
|
|
|
|
+
|
|
|
|
+ INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
|
|
|
|
+ BlockInfo[] blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(1, blks.length);
|
|
|
|
+ assertEquals(BLOCKSIZE, blks[0].getNumBytes());
|
|
|
|
+ ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
|
|
|
|
+ cluster.getNameNodeRpc()
|
|
|
|
+ .addBlock(bar.toString(), hdfs.getClient().getClientName(), previous,
|
|
|
|
+ null, barNode.getId(), null);
|
|
|
|
+
|
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, foo, "s1");
|
|
|
|
+
|
|
|
|
+ barNode = fsdir.getINode4Write(bar.toString()).asFile();
|
|
|
|
+ blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(2, blks.length);
|
|
|
|
+ assertEquals(BLOCKSIZE, blks[0].getNumBytes());
|
|
|
|
+ assertEquals(0, blks[1].getNumBytes());
|
|
|
|
+
|
|
|
|
+ hdfs.delete(bar, true);
|
|
|
|
+ final Path sbar = SnapshotTestHelper.getSnapshotPath(foo, "s1",
|
|
|
|
+ bar.getName());
|
|
|
|
+ barNode = fsdir.getINode(sbar.toString()).asFile();
|
|
|
|
+ blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(1, blks.length);
|
|
|
|
+ assertEquals(BLOCKSIZE, blks[0].getNumBytes());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Make sure we delete 0-sized block when deleting an INodeFileUC */
|
|
|
|
+ @Test
|
|
|
|
+ public void testDeletionWithZeroSizeBlock2() throws Exception {
|
|
|
|
+ final Path foo = new Path("/foo");
|
|
|
|
+ final Path subDir = new Path(foo, "sub");
|
|
|
|
+ final Path bar = new Path(subDir, "bar");
|
|
|
|
+ DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPLICATION, 0L);
|
|
|
|
+
|
|
|
|
+ hdfs.append(bar);
|
|
|
|
+
|
|
|
|
+ INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
|
|
|
|
+ BlockInfo[] blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(1, blks.length);
|
|
|
|
+ ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
|
|
|
|
+ cluster.getNameNodeRpc()
|
|
|
|
+ .addBlock(bar.toString(), hdfs.getClient().getClientName(), previous,
|
|
|
|
+ null, barNode.getId(), null);
|
|
|
|
+
|
|
|
|
+ SnapshotTestHelper.createSnapshot(hdfs, foo, "s1");
|
|
|
|
+
|
|
|
|
+ barNode = fsdir.getINode4Write(bar.toString()).asFile();
|
|
|
|
+ blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(2, blks.length);
|
|
|
|
+ assertEquals(BLOCKSIZE, blks[0].getNumBytes());
|
|
|
|
+ assertEquals(0, blks[1].getNumBytes());
|
|
|
|
+
|
|
|
|
+ hdfs.delete(subDir, true);
|
|
|
|
+ final Path sbar = SnapshotTestHelper.getSnapshotPath(foo, "s1", "sub/bar");
|
|
|
|
+ barNode = fsdir.getINode(sbar.toString()).asFile();
|
|
|
|
+ blks = barNode.getBlocks();
|
|
|
|
+ assertEquals(1, blks.length);
|
|
|
|
+ assertEquals(BLOCKSIZE, blks[0].getNumBytes());
|
|
|
|
+ }
|
|
}
|
|
}
|