|
@@ -17,19 +17,21 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.fs.StorageType;
|
|
|
-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.server.common.HdfsServerConstants;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
|
|
-import org.junit.After;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
+import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
|
|
|
import org.junit.Assert;
|
|
|
-import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.internal.util.reflection.Whitebox;
|
|
|
+
|
|
|
+import java.util.ArrayList;
|
|
|
+
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
/**
|
|
|
* Make sure we correctly update the quota usage for truncate.
|
|
@@ -45,204 +47,119 @@ import org.junit.Test;
|
|
|
public class TestTruncateQuotaUpdate {
|
|
|
private static final int BLOCKSIZE = 1024;
|
|
|
private static final short REPLICATION = 4;
|
|
|
- private static final long DISKQUOTA = BLOCKSIZE * 20;
|
|
|
- static final long seed = 0L;
|
|
|
- private static final Path dir = new Path("/TestTruncateQuotaUpdate");
|
|
|
- private static final Path file = new Path(dir, "file");
|
|
|
-
|
|
|
- private MiniDFSCluster cluster;
|
|
|
- private FSDirectory fsdir;
|
|
|
- private DistributedFileSystem dfs;
|
|
|
-
|
|
|
- @Before
|
|
|
- public void setUp() throws Exception {
|
|
|
- final Configuration conf = new Configuration();
|
|
|
- conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
|
|
|
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
|
|
|
- .build();
|
|
|
- cluster.waitActive();
|
|
|
-
|
|
|
- fsdir = cluster.getNamesystem().getFSDirectory();
|
|
|
- dfs = cluster.getFileSystem();
|
|
|
-
|
|
|
- dfs.mkdirs(dir);
|
|
|
- dfs.setQuota(dir, Long.MAX_VALUE - 1, DISKQUOTA);
|
|
|
- dfs.setQuotaByStorageType(dir, StorageType.DISK, DISKQUOTA);
|
|
|
- dfs.setStoragePolicy(dir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
|
|
|
- }
|
|
|
-
|
|
|
- @After
|
|
|
- public void tearDown() throws Exception {
|
|
|
- if (cluster != null) {
|
|
|
- cluster.shutdown();
|
|
|
- }
|
|
|
- }
|
|
|
+ private long nextMockBlockId;
|
|
|
+ private long nextMockGenstamp;
|
|
|
+ private long nextMockINodeId;
|
|
|
|
|
|
@Test
|
|
|
- public void testTruncateQuotaUpdate() throws Exception {
|
|
|
-
|
|
|
+ public void testTruncateWithoutSnapshot() {
|
|
|
+ INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
|
|
|
+ // case 1: first truncate to 1.5 blocks
|
|
|
+ // we truncate 1 blocks, but not on the boundary, thus the diff should
|
|
|
+ // be -block + (block - 0.5 block) = -0.5 block
|
|
|
+ QuotaCounts count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE + BLOCKSIZE / 2, null, count);
|
|
|
+ Assert.assertEquals(-BLOCKSIZE / 2 * REPLICATION, count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 2: truncate to 1 block
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE, null, count);
|
|
|
+ Assert.assertEquals(-(BLOCKSIZE + BLOCKSIZE / 2) * REPLICATION,
|
|
|
+ count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 3: truncate to 0
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(0, null, count);
|
|
|
+ Assert.assertEquals(-(BLOCKSIZE * 2 + BLOCKSIZE / 2) * REPLICATION,
|
|
|
+ count.getStorageSpace());
|
|
|
}
|
|
|
|
|
|
- public interface TruncateCase {
|
|
|
- public void prepare() throws Exception;
|
|
|
- public void run() throws Exception;
|
|
|
- }
|
|
|
-
|
|
|
- private void testTruncate(long newLength, long expectedDiff,
|
|
|
- long expectedUsage) throws Exception {
|
|
|
- // before doing the real truncation, make sure the computation is correct
|
|
|
- final INodesInPath iip = fsdir.getINodesInPath4Write(file.toString());
|
|
|
- final INodeFile fileNode = iip.getLastINode().asFile();
|
|
|
- fileNode.recordModification(iip.getLatestSnapshotId(), true);
|
|
|
- final long diff = fileNode.computeQuotaDeltaForTruncate(newLength);
|
|
|
- Assert.assertEquals(expectedDiff, diff);
|
|
|
-
|
|
|
- // do the real truncation
|
|
|
- dfs.truncate(file, newLength);
|
|
|
- // wait for truncate to finish
|
|
|
- TestFileTruncate.checkBlockRecovery(file, dfs);
|
|
|
- final INodeDirectory dirNode = fsdir.getINode4Write(dir.toString())
|
|
|
- .asDirectory();
|
|
|
- final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
- .getSpaceConsumed().getStorageSpace();
|
|
|
- final long diskUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
- .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
|
|
|
- Assert.assertEquals(expectedUsage, spaceUsed);
|
|
|
- Assert.assertEquals(expectedUsage, diskUsed);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * case 1~3
|
|
|
- */
|
|
|
- private class TruncateWithoutSnapshot implements TruncateCase {
|
|
|
- @Override
|
|
|
- public void prepare() throws Exception {
|
|
|
- // original file size: 2.5 block
|
|
|
- DFSTestUtil.createFile(dfs, file, BLOCKSIZE * 2 + BLOCKSIZE / 2,
|
|
|
- REPLICATION, 0L);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void run() throws Exception {
|
|
|
- // case 1: first truncate to 1.5 blocks
|
|
|
- long newLength = BLOCKSIZE + BLOCKSIZE / 2;
|
|
|
- // we truncate 1 blocks, but not on the boundary, thus the diff should
|
|
|
- // be -block + (block - 0.5 block) = -0.5 block
|
|
|
- long diff = -BLOCKSIZE / 2;
|
|
|
- // the new quota usage should be BLOCKSIZE * 1.5 * replication
|
|
|
- long usage = (BLOCKSIZE + BLOCKSIZE / 2) * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 2: truncate to 1 block
|
|
|
- newLength = BLOCKSIZE;
|
|
|
- // the diff should be -0.5 block since this is not on boundary
|
|
|
- diff = -BLOCKSIZE / 2;
|
|
|
- // after truncation the quota usage should be BLOCKSIZE * replication
|
|
|
- usage = BLOCKSIZE * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 3: truncate to 0
|
|
|
- testTruncate(0, -BLOCKSIZE, 0);
|
|
|
- }
|
|
|
+ @Test
|
|
|
+ public void testTruncateWithSnapshotNoDivergence() {
|
|
|
+ INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
|
|
|
+ addSnapshotFeature(file, file.getBlocks());
|
|
|
+
|
|
|
+ // case 4: truncate to 1.5 blocks
|
|
|
+ // all the blocks are in snapshot. truncate need to allocate a new block
|
|
|
+ // diff should be +BLOCKSIZE
|
|
|
+ QuotaCounts count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE + BLOCKSIZE / 2, null, count);
|
|
|
+ Assert.assertEquals(BLOCKSIZE * REPLICATION, count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 2: truncate to 1 block
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE, null, count);
|
|
|
+ Assert.assertEquals(0, count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 3: truncate to 0
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(0, null, count);
|
|
|
+ Assert.assertEquals(0, count.getStorageSpace());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * case 4~6
|
|
|
- */
|
|
|
- private class TruncateWithSnapshot implements TruncateCase {
|
|
|
- @Override
|
|
|
- public void prepare() throws Exception {
|
|
|
- DFSTestUtil.createFile(dfs, file, BLOCKSIZE * 2 + BLOCKSIZE / 2,
|
|
|
- REPLICATION, 0L);
|
|
|
- SnapshotTestHelper.createSnapshot(dfs, dir, "s1");
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void run() throws Exception {
|
|
|
- // case 4: truncate to 1.5 blocks
|
|
|
- long newLength = BLOCKSIZE + BLOCKSIZE / 2;
|
|
|
- // all the blocks are in snapshot. truncate need to allocate a new block
|
|
|
- // diff should be +BLOCKSIZE
|
|
|
- long diff = BLOCKSIZE;
|
|
|
- // the new quota usage should be BLOCKSIZE * 3 * replication
|
|
|
- long usage = BLOCKSIZE * 3 * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 5: truncate to 1 block
|
|
|
- newLength = BLOCKSIZE;
|
|
|
- // the block for truncation is not in snapshot, diff should be -0.5 block
|
|
|
- diff = -BLOCKSIZE / 2;
|
|
|
- // after truncation the quota usage should be 2.5 block * repl
|
|
|
- usage = (BLOCKSIZE * 2 + BLOCKSIZE / 2) * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 6: truncate to 0
|
|
|
- testTruncate(0, 0, usage);
|
|
|
- }
|
|
|
+ @Test
|
|
|
+ public void testTruncateWithSnapshotAndDivergence() {
|
|
|
+ INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
|
|
|
+ BlockInfoContiguous[] blocks = new BlockInfoContiguous
|
|
|
+ [file.getBlocks().length];
|
|
|
+ System.arraycopy(file.getBlocks(), 0, blocks, 0, blocks.length);
|
|
|
+ addSnapshotFeature(file, blocks);
|
|
|
+ // Update the last two blocks in the current inode
|
|
|
+ file.getBlocks()[1] = newBlock(BLOCKSIZE, REPLICATION);
|
|
|
+ file.getBlocks()[2] = newBlock(BLOCKSIZE / 2, REPLICATION);
|
|
|
+
|
|
|
+ // case 7: truncate to 1.5 block
|
|
|
+ // the block for truncation is not in snapshot, diff should be the same
|
|
|
+ // as case 1
|
|
|
+ QuotaCounts count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE + BLOCKSIZE / 2, null, count);
|
|
|
+ Assert.assertEquals(-BLOCKSIZE / 2 * REPLICATION, count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 8: truncate to 2 blocks
|
|
|
+ // the original 2.5 blocks are in snapshot. the block truncated is not
|
|
|
+ // in snapshot. diff should be -0.5 block
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(BLOCKSIZE + BLOCKSIZE / 2, null, count);
|
|
|
+ Assert.assertEquals(-BLOCKSIZE / 2 * REPLICATION, count.getStorageSpace());
|
|
|
+
|
|
|
+ // case 9: truncate to 0
|
|
|
+ count = new QuotaCounts.Builder().build();
|
|
|
+ file.computeQuotaDeltaForTruncate(0, null, count);
|
|
|
+ Assert.assertEquals(-(BLOCKSIZE + BLOCKSIZE / 2) * REPLICATION, count
|
|
|
+ .getStorageSpace());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * case 7~9
|
|
|
- */
|
|
|
- private class TruncateWithSnapshot2 implements TruncateCase {
|
|
|
- @Override
|
|
|
- public void prepare() throws Exception {
|
|
|
- // original size: 2.5 blocks
|
|
|
- DFSTestUtil.createFile(dfs, file, BLOCKSIZE * 2 + BLOCKSIZE / 2,
|
|
|
- REPLICATION, 0L);
|
|
|
- SnapshotTestHelper.createSnapshot(dfs, dir, "s1");
|
|
|
-
|
|
|
- // truncate to 1.5 block
|
|
|
- dfs.truncate(file, BLOCKSIZE + BLOCKSIZE / 2);
|
|
|
- TestFileTruncate.checkBlockRecovery(file, dfs);
|
|
|
-
|
|
|
- // append another 1 BLOCK
|
|
|
- DFSTestUtil.appendFile(dfs, file, BLOCKSIZE);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void run() throws Exception {
|
|
|
- // case 8: truncate to 2 blocks
|
|
|
- long newLength = BLOCKSIZE * 2;
|
|
|
- // the original 2.5 blocks are in snapshot. the block truncated is not
|
|
|
- // in snapshot. diff should be -0.5 block
|
|
|
- long diff = -BLOCKSIZE / 2;
|
|
|
- // the new quota usage should be BLOCKSIZE * 3.5 * replication
|
|
|
- long usage = (BLOCKSIZE * 3 + BLOCKSIZE / 2) * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 7: truncate to 1.5 block
|
|
|
- newLength = BLOCKSIZE + BLOCKSIZE / 2;
|
|
|
- // the block for truncation is not in snapshot, diff should be
|
|
|
- // -0.5 block + (block - 0.5block) = 0
|
|
|
- diff = 0;
|
|
|
- // after truncation the quota usage should be 3 block * repl
|
|
|
- usage = (BLOCKSIZE * 3) * REPLICATION;
|
|
|
- testTruncate(newLength, diff, usage);
|
|
|
-
|
|
|
- // case 9: truncate to 0
|
|
|
- testTruncate(0, -BLOCKSIZE / 2,
|
|
|
- (BLOCKSIZE * 2 + BLOCKSIZE / 2) * REPLICATION);
|
|
|
+ private INodeFile createMockFile(long size, short replication) {
|
|
|
+ ArrayList<BlockInfoContiguous> blocks = new ArrayList<>();
|
|
|
+ long createdSize = 0;
|
|
|
+ while (createdSize < size) {
|
|
|
+ long blockSize = Math.min(BLOCKSIZE, size - createdSize);
|
|
|
+ BlockInfoContiguous bi = newBlock(blockSize, replication);
|
|
|
+ blocks.add(bi);
|
|
|
+ createdSize += BLOCKSIZE;
|
|
|
}
|
|
|
+ PermissionStatus perm = new PermissionStatus("foo", "bar", FsPermission
|
|
|
+ .createImmutable((short) 0x1ff));
|
|
|
+ return new INodeFile(
|
|
|
+ ++nextMockINodeId, new byte[0], perm, 0, 0,
|
|
|
+ blocks.toArray(new BlockInfoContiguous[blocks.size()]), replication,
|
|
|
+ BLOCKSIZE);
|
|
|
}
|
|
|
|
|
|
- private void testTruncateQuotaUpdate(TruncateCase t) throws Exception {
|
|
|
- t.prepare();
|
|
|
- t.run();
|
|
|
+ private BlockInfoContiguous newBlock(long size, short replication) {
|
|
|
+ Block b = new Block(++nextMockBlockId, size, ++nextMockGenstamp);
|
|
|
+ return new BlockInfoContiguous(b, replication);
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testQuotaNoSnapshot() throws Exception {
|
|
|
- testTruncateQuotaUpdate(new TruncateWithoutSnapshot());
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testQuotaWithSnapshot() throws Exception {
|
|
|
- testTruncateQuotaUpdate(new TruncateWithSnapshot());
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testQuotaWithSnapshot2() throws Exception {
|
|
|
- testTruncateQuotaUpdate(new TruncateWithSnapshot2());
|
|
|
+ private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) {
|
|
|
+ FileDiff diff = mock(FileDiff.class);
|
|
|
+ when(diff.getBlocks()).thenReturn(blocks);
|
|
|
+ FileDiffList diffList = new FileDiffList();
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ ArrayList<FileDiff> diffs = ((ArrayList<FileDiff>)Whitebox.getInternalState
|
|
|
+ (diffList, "diffs"));
|
|
|
+ diffs.add(diff);
|
|
|
+ FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffList);
|
|
|
+ file.addFeature(sf);
|
|
|
}
|
|
|
}
|