|
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
@@ -93,6 +94,8 @@ public class TestFileTruncate {
|
|
|
conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BLOCK_SIZE);
|
|
|
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, SHORT_HEARTBEAT);
|
|
|
+ conf.setLong(
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 1);
|
|
|
cluster = new MiniDFSCluster.Builder(conf)
|
|
|
.format(true)
|
|
|
.numDataNodes(DATANODE_NUM)
|
|
@@ -622,6 +625,224 @@ public class TestFileTruncate {
|
|
|
fs.delete(p, false);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The last block is truncated at mid. (non copy-on-truncate)
|
|
|
+ * dn0 is shutdown before truncate and restart after truncate successful.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testTruncateWithDataNodesRestart() throws Exception {
|
|
|
+ int startingFileSize = 3 * BLOCK_SIZE;
|
|
|
+ byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
|
+ final Path parent = new Path("/test");
|
|
|
+ final Path p = new Path(parent, "testTruncateWithDataNodesRestart");
|
|
|
+
|
|
|
+ writeContents(contents, startingFileSize, p);
|
|
|
+ LocatedBlock oldBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+
|
|
|
+ int dn = 0;
|
|
|
+ int toTruncateLength = 1;
|
|
|
+ int newLength = startingFileSize - toTruncateLength;
|
|
|
+ cluster.getDataNodes().get(dn).shutdown();
|
|
|
+ try {
|
|
|
+ boolean isReady = fs.truncate(p, newLength);
|
|
|
+ assertFalse(isReady);
|
|
|
+ } finally {
|
|
|
+ cluster.restartDataNode(dn);
|
|
|
+ cluster.waitActive();
|
|
|
+ cluster.triggerBlockReports();
|
|
|
+ }
|
|
|
+
|
|
|
+ LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+ /*
|
|
|
+ * For non copy-on-truncate, the truncated block id is the same, but the
|
|
|
+ * GS should increase.
|
|
|
+ * We trigger block report for dn0 after it restarts, since the GS
|
|
|
+ * of replica for the last block on it is old, so the reported last block
|
|
|
+ * from dn0 should be marked corrupt on nn and the replicas of last block
|
|
|
+ * on nn should decrease 1, then the truncated block will be replicated
|
|
|
+ * to dn0.
|
|
|
+ */
|
|
|
+ assertEquals(newBlock.getBlock().getBlockId(),
|
|
|
+ oldBlock.getBlock().getBlockId());
|
|
|
+ assertEquals(newBlock.getBlock().getGenerationStamp(),
|
|
|
+ oldBlock.getBlock().getGenerationStamp() + 1);
|
|
|
+
|
|
|
+ checkBlockRecovery(p);
|
|
|
+ // Wait replicas come to 3
|
|
|
+ DFSTestUtil.waitReplication(fs, p, REPLICATION);
|
|
|
+ // Old replica is disregarded and replaced with the truncated one
|
|
|
+ assertEquals(cluster.getBlockFile(dn, newBlock.getBlock()).length(),
|
|
|
+ newBlock.getBlockSize());
|
|
|
+ assertTrue(cluster.getBlockMetadataFile(dn,
|
|
|
+ newBlock.getBlock()).getName().endsWith(
|
|
|
+ newBlock.getBlock().getGenerationStamp() + ".meta"));
|
|
|
+
|
|
|
+ // Validate the file
|
|
|
+ FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
+ assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
+ checkFullFile(p, newLength, contents);
|
|
|
+
|
|
|
+ fs.delete(parent, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The last block is truncated at mid. (copy-on-truncate)
|
|
|
+ * dn1 is shutdown before truncate and restart after truncate successful.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testCopyOnTruncateWithDataNodesRestart() throws Exception {
|
|
|
+ int startingFileSize = 3 * BLOCK_SIZE;
|
|
|
+ byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
|
+ final Path parent = new Path("/test");
|
|
|
+ final Path p = new Path(parent, "testCopyOnTruncateWithDataNodesRestart");
|
|
|
+
|
|
|
+ writeContents(contents, startingFileSize, p);
|
|
|
+ LocatedBlock oldBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+ fs.allowSnapshot(parent);
|
|
|
+ fs.createSnapshot(parent, "ss0");
|
|
|
+
|
|
|
+ int dn = 1;
|
|
|
+ int toTruncateLength = 1;
|
|
|
+ int newLength = startingFileSize - toTruncateLength;
|
|
|
+ cluster.getDataNodes().get(dn).shutdown();
|
|
|
+ try {
|
|
|
+ boolean isReady = fs.truncate(p, newLength);
|
|
|
+ assertFalse(isReady);
|
|
|
+ } finally {
|
|
|
+ cluster.restartDataNode(dn);
|
|
|
+ cluster.waitActive();
|
|
|
+ cluster.triggerBlockReports();
|
|
|
+ }
|
|
|
+
|
|
|
+ LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+ /*
|
|
|
+ * For copy-on-truncate, new block is made with new block id and new GS.
|
|
|
+ * We trigger block report for dn1 after it restarts. The replicas of
|
|
|
+ * the new block is 2, and then it will be replicated to dn1.
|
|
|
+ */
|
|
|
+ assertNotEquals(newBlock.getBlock().getBlockId(),
|
|
|
+ oldBlock.getBlock().getBlockId());
|
|
|
+ assertEquals(newBlock.getBlock().getGenerationStamp(),
|
|
|
+ oldBlock.getBlock().getGenerationStamp() + 1);
|
|
|
+
|
|
|
+ checkBlockRecovery(p);
|
|
|
+ // Wait replicas come to 3
|
|
|
+ DFSTestUtil.waitReplication(fs, p, REPLICATION);
|
|
|
+ // New block is replicated to dn1
|
|
|
+ assertEquals(cluster.getBlockFile(dn, newBlock.getBlock()).length(),
|
|
|
+ newBlock.getBlockSize());
|
|
|
+ // Old replica exists too since there is snapshot
|
|
|
+ assertEquals(cluster.getBlockFile(dn, oldBlock.getBlock()).length(),
|
|
|
+ oldBlock.getBlockSize());
|
|
|
+ assertTrue(cluster.getBlockMetadataFile(dn,
|
|
|
+ oldBlock.getBlock()).getName().endsWith(
|
|
|
+ oldBlock.getBlock().getGenerationStamp() + ".meta"));
|
|
|
+
|
|
|
+ // Validate the file
|
|
|
+ FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
+ assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
+ checkFullFile(p, newLength, contents);
|
|
|
+
|
|
|
+ fs.deleteSnapshot(parent, "ss0");
|
|
|
+ fs.delete(parent, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The last block is truncated at mid. (non copy-on-truncate)
|
|
|
+ * dn0, dn1 are restarted immediately after truncate.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testTruncateWithDataNodesRestartImmediately() throws Exception {
|
|
|
+ int startingFileSize = 3 * BLOCK_SIZE;
|
|
|
+ byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
|
+ final Path parent = new Path("/test");
|
|
|
+ final Path p = new Path(parent, "testTruncateWithDataNodesRestartImmediately");
|
|
|
+
|
|
|
+ writeContents(contents, startingFileSize, p);
|
|
|
+ LocatedBlock oldBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+
|
|
|
+ int dn0 = 0;
|
|
|
+ int dn1 = 1;
|
|
|
+ int toTruncateLength = 1;
|
|
|
+ int newLength = startingFileSize - toTruncateLength;
|
|
|
+ boolean isReady = fs.truncate(p, newLength);
|
|
|
+ assertFalse(isReady);
|
|
|
+
|
|
|
+ cluster.restartDataNode(dn0);
|
|
|
+ cluster.restartDataNode(dn1);
|
|
|
+ cluster.waitActive();
|
|
|
+ cluster.triggerBlockReports();
|
|
|
+
|
|
|
+ LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
|
|
|
+ /*
|
|
|
+ * For non copy-on-truncate, the truncated block id is the same, but the
|
|
|
+ * GS should increase.
|
|
|
+ */
|
|
|
+ assertEquals(newBlock.getBlock().getBlockId(),
|
|
|
+ oldBlock.getBlock().getBlockId());
|
|
|
+ assertEquals(newBlock.getBlock().getGenerationStamp(),
|
|
|
+ oldBlock.getBlock().getGenerationStamp() + 1);
|
|
|
+
|
|
|
+ checkBlockRecovery(p);
|
|
|
+ // Wait replicas come to 3
|
|
|
+ DFSTestUtil.waitReplication(fs, p, REPLICATION);
|
|
|
+ // Old replica is disregarded and replaced with the truncated one on dn0
|
|
|
+ assertEquals(cluster.getBlockFile(dn0, newBlock.getBlock()).length(),
|
|
|
+ newBlock.getBlockSize());
|
|
|
+ assertTrue(cluster.getBlockMetadataFile(dn0,
|
|
|
+ newBlock.getBlock()).getName().endsWith(
|
|
|
+ newBlock.getBlock().getGenerationStamp() + ".meta"));
|
|
|
+
|
|
|
+ // Old replica is disregarded and replaced with the truncated one on dn1
|
|
|
+ assertEquals(cluster.getBlockFile(dn1, newBlock.getBlock()).length(),
|
|
|
+ newBlock.getBlockSize());
|
|
|
+ assertTrue(cluster.getBlockMetadataFile(dn1,
|
|
|
+ newBlock.getBlock()).getName().endsWith(
|
|
|
+ newBlock.getBlock().getGenerationStamp() + ".meta"));
|
|
|
+
|
|
|
+ // Validate the file
|
|
|
+ FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
+ assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
+ checkFullFile(p, newLength, contents);
|
|
|
+
|
|
|
+ fs.delete(parent, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The last block is truncated at mid. (non copy-on-truncate)
|
|
|
+ * shutdown the datanodes immediately after truncate.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testTruncateWithDataNodesShutdownImmediately() throws Exception {
|
|
|
+ int startingFileSize = 3 * BLOCK_SIZE;
|
|
|
+ byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
|
+ final Path parent = new Path("/test");
|
|
|
+ final Path p = new Path(parent, "testTruncateWithDataNodesShutdownImmediately");
|
|
|
+
|
|
|
+ writeContents(contents, startingFileSize, p);
|
|
|
+
|
|
|
+ int toTruncateLength = 1;
|
|
|
+ int newLength = startingFileSize - toTruncateLength;
|
|
|
+ boolean isReady = fs.truncate(p, newLength);
|
|
|
+ assertFalse(isReady);
|
|
|
+
|
|
|
+ cluster.shutdownDataNodes();
|
|
|
+ try {
|
|
|
+ for(int i = 0; i < SUCCESS_ATTEMPTS && cluster.isDataNodeUp(); i++) {
|
|
|
+ Thread.sleep(SLEEP);
|
|
|
+ }
|
|
|
+ assertFalse("All DataNodes should be down.", cluster.isDataNodeUp());
|
|
|
+ LocatedBlocks blocks = getLocatedBlocks(p);
|
|
|
+ assertTrue(blocks.isUnderConstruction());
|
|
|
+ } finally {
|
|
|
+ cluster.startDataNodes(conf, DATANODE_NUM, true,
|
|
|
+ StartupOption.REGULAR, null);
|
|
|
+ cluster.waitActive();
|
|
|
+ }
|
|
|
+
|
|
|
+ fs.delete(parent, true);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* EditLogOp load test for Truncate.
|
|
|
*/
|