|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
import static org.hamcrest.core.Is.is;
|
|
|
+import static org.junit.Assert.assertArrayEquals;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
@@ -28,6 +29,8 @@ import static org.junit.Assert.assertTrue;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.Random;
|
|
|
+import java.util.regex.Matcher;
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -47,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
/**
|
|
@@ -64,7 +68,7 @@ public class TestDataNodeRollingUpgrade {
|
|
|
Configuration conf;
|
|
|
MiniDFSCluster cluster = null;
|
|
|
DistributedFileSystem fs = null;
|
|
|
- DataNode dn = null;
|
|
|
+ DataNode dn0 = null;
|
|
|
NameNode nn = null;
|
|
|
String blockPoolId = null;
|
|
|
|
|
@@ -76,8 +80,8 @@ public class TestDataNodeRollingUpgrade {
|
|
|
fs = cluster.getFileSystem();
|
|
|
nn = cluster.getNameNode(0);
|
|
|
assertNotNull(nn);
|
|
|
- dn = cluster.getDataNodes().get(0);
|
|
|
- assertNotNull(dn);
|
|
|
+ dn0 = cluster.getDataNodes().get(0);
|
|
|
+ assertNotNull(dn0);
|
|
|
blockPoolId = cluster.getNameNode(0).getNamesystem().getBlockPoolId();
|
|
|
}
|
|
|
|
|
@@ -88,7 +92,7 @@ public class TestDataNodeRollingUpgrade {
|
|
|
}
|
|
|
fs = null;
|
|
|
nn = null;
|
|
|
- dn = null;
|
|
|
+ dn0 = null;
|
|
|
blockPoolId = null;
|
|
|
}
|
|
|
|
|
@@ -103,9 +107,10 @@ public class TestDataNodeRollingUpgrade {
|
|
|
private File getBlockForFile(Path path, boolean exists) throws IOException {
|
|
|
LocatedBlocks blocks = nn.getRpcServer().getBlockLocations(path.toString(),
|
|
|
0, Long.MAX_VALUE);
|
|
|
- assertEquals(1, blocks.getLocatedBlocks().size());
|
|
|
+ assertEquals("The test helper functions assume that each file has a single block",
|
|
|
+ 1, blocks.getLocatedBlocks().size());
|
|
|
ExtendedBlock block = blocks.getLocatedBlocks().get(0).getBlock();
|
|
|
- BlockLocalPathInfo bInfo = dn.getFSDataset().getBlockLocalPathInfo(block);
|
|
|
+ BlockLocalPathInfo bInfo = dn0.getFSDataset().getBlockLocalPathInfo(block);
|
|
|
File blockFile = new File(bInfo.getBlockPath());
|
|
|
assertEquals(exists, blockFile.exists());
|
|
|
return blockFile;
|
|
@@ -113,7 +118,7 @@ public class TestDataNodeRollingUpgrade {
|
|
|
|
|
|
private File getTrashFileForBlock(File blockFile, boolean exists) {
|
|
|
File trashFile = new File(
|
|
|
- dn.getStorage().getTrashDirectoryForBlockFile(blockPoolId, blockFile));
|
|
|
+ dn0.getStorage().getTrashDirectoryForBlockFile(blockPoolId, blockFile));
|
|
|
assertEquals(exists, trashFile.exists());
|
|
|
return trashFile;
|
|
|
}
|
|
@@ -135,11 +140,10 @@ public class TestDataNodeRollingUpgrade {
|
|
|
assertFalse(blockFile.exists());
|
|
|
}
|
|
|
|
|
|
- private void ensureTrashDisabled() {
|
|
|
+ private boolean isTrashRootPresent() {
|
|
|
// Trash is disabled; trash root does not exist
|
|
|
- assertFalse(dn.getFSDataset().trashEnabled(blockPoolId));
|
|
|
- BlockPoolSliceStorage bps = dn.getStorage().getBPStorage(blockPoolId);
|
|
|
- assertFalse(bps.trashEnabled());
|
|
|
+ BlockPoolSliceStorage bps = dn0.getStorage().getBPStorage(blockPoolId);
|
|
|
+ return bps.trashEnabled();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -149,17 +153,25 @@ public class TestDataNodeRollingUpgrade {
|
|
|
throws Exception {
|
|
|
assertTrue(blockFile.exists());
|
|
|
assertFalse(trashFile.exists());
|
|
|
- ensureTrashDisabled();
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean isBlockFileInPrevious(File blockFile) {
|
|
|
+ Pattern blockFilePattern = Pattern.compile("^(.*/current/.*/)(current)(/.*)$");
|
|
|
+ Matcher matcher = blockFilePattern.matcher(blockFile.toString());
|
|
|
+ String previousFileName = matcher.replaceFirst("$1" + "previous" + "$3");
|
|
|
+ return ((new File(previousFileName)).exists());
|
|
|
}
|
|
|
|
|
|
private void startRollingUpgrade() throws Exception {
|
|
|
LOG.info("Starting rolling upgrade");
|
|
|
+ fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
|
|
final DFSAdmin dfsadmin = new DFSAdmin(conf);
|
|
|
TestRollingUpgrade.runCmd(dfsadmin, true, "-rollingUpgrade", "prepare");
|
|
|
triggerHeartBeats();
|
|
|
|
|
|
// Ensure datanode rolling upgrade is started
|
|
|
- assertTrue(dn.getFSDataset().trashEnabled(blockPoolId));
|
|
|
+ assertTrue(dn0.getFSDataset().trashEnabled(blockPoolId));
|
|
|
}
|
|
|
|
|
|
private void finalizeRollingUpgrade() throws Exception {
|
|
@@ -169,8 +181,8 @@ public class TestDataNodeRollingUpgrade {
|
|
|
triggerHeartBeats();
|
|
|
|
|
|
// Ensure datanode rolling upgrade is started
|
|
|
- assertFalse(dn.getFSDataset().trashEnabled(blockPoolId));
|
|
|
- BlockPoolSliceStorage bps = dn.getStorage().getBPStorage(blockPoolId);
|
|
|
+ assertFalse(dn0.getFSDataset().trashEnabled(blockPoolId));
|
|
|
+ BlockPoolSliceStorage bps = dn0.getStorage().getBPStorage(blockPoolId);
|
|
|
assertFalse(bps.trashEnabled());
|
|
|
}
|
|
|
|
|
@@ -179,13 +191,15 @@ public class TestDataNodeRollingUpgrade {
|
|
|
// Restart the namenode with rolling upgrade rollback
|
|
|
LOG.info("Starting rollback of the rolling upgrade");
|
|
|
MiniDFSCluster.DataNodeProperties dnprop = cluster.stopDataNode(0);
|
|
|
+ dnprop.setDnArgs("-rollback");
|
|
|
cluster.shutdownNameNodes();
|
|
|
cluster.restartNameNode("-rollingupgrade", "rollback");
|
|
|
cluster.restartDataNode(dnprop);
|
|
|
cluster.waitActive();
|
|
|
nn = cluster.getNameNode(0);
|
|
|
- dn = cluster.getDataNodes().get(0);
|
|
|
+ dn0 = cluster.getDataNodes().get(0);
|
|
|
triggerHeartBeats();
|
|
|
+ LOG.info("The cluster is active after rollback");
|
|
|
}
|
|
|
|
|
|
@Test (timeout=600000)
|
|
@@ -194,12 +208,11 @@ public class TestDataNodeRollingUpgrade {
|
|
|
startCluster();
|
|
|
|
|
|
// Create files in DFS.
|
|
|
- Path testFile1 = new Path("/TestDataNodeRollingUpgrade1.dat");
|
|
|
- Path testFile2 = new Path("/TestDataNodeRollingUpgrade2.dat");
|
|
|
+ Path testFile1 = new Path("/" + GenericTestUtils.getMethodName() + ".01.dat");
|
|
|
+ Path testFile2 = new Path("/" + GenericTestUtils.getMethodName() + ".02.dat");
|
|
|
DFSTestUtil.createFile(fs, testFile1, FILE_SIZE, REPL_FACTOR, SEED);
|
|
|
DFSTestUtil.createFile(fs, testFile2, FILE_SIZE, REPL_FACTOR, SEED);
|
|
|
|
|
|
- fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
|
|
startRollingUpgrade();
|
|
|
File blockFile = getBlockForFile(testFile2, true);
|
|
|
File trashFile = getTrashFileForBlock(blockFile, false);
|
|
@@ -207,7 +220,7 @@ public class TestDataNodeRollingUpgrade {
|
|
|
finalizeRollingUpgrade();
|
|
|
|
|
|
// Ensure that delete file testFile2 stays deleted after finalize
|
|
|
- ensureTrashDisabled();
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
assert(!fs.exists(testFile2));
|
|
|
assert(fs.exists(testFile1));
|
|
|
|
|
@@ -222,11 +235,10 @@ public class TestDataNodeRollingUpgrade {
|
|
|
startCluster();
|
|
|
|
|
|
// Create files in DFS.
|
|
|
- Path testFile1 = new Path("/TestDataNodeRollingUpgrade1.dat");
|
|
|
+ Path testFile1 = new Path("/" + GenericTestUtils.getMethodName() + ".01.dat");
|
|
|
DFSTestUtil.createFile(fs, testFile1, FILE_SIZE, REPL_FACTOR, SEED);
|
|
|
String fileContents1 = DFSTestUtil.readFile(fs, testFile1);
|
|
|
|
|
|
- fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
|
|
startRollingUpgrade();
|
|
|
|
|
|
File blockFile = getBlockForFile(testFile1, true);
|
|
@@ -255,9 +267,9 @@ public class TestDataNodeRollingUpgrade {
|
|
|
startCluster();
|
|
|
|
|
|
// Create files in DFS.
|
|
|
- String testFile1 = "/TestDataNodeXceiver1.dat";
|
|
|
- String testFile2 = "/TestDataNodeXceiver2.dat";
|
|
|
- String testFile3 = "/TestDataNodeXceiver3.dat";
|
|
|
+ String testFile1 = "/" + GenericTestUtils.getMethodName() + ".01.dat";
|
|
|
+ String testFile2 = "/" + GenericTestUtils.getMethodName() + ".02.dat";
|
|
|
+ String testFile3 = "/" + GenericTestUtils.getMethodName() + ".03.dat";
|
|
|
|
|
|
DFSClient client1 = new DFSClient(NameNode.getAddress(conf), conf);
|
|
|
DFSClient client2 = new DFSClient(NameNode.getAddress(conf), conf);
|
|
@@ -277,12 +289,12 @@ public class TestDataNodeRollingUpgrade {
|
|
|
s3.write(toWrite, 0, 1024*1024*8);
|
|
|
s3.flush();
|
|
|
|
|
|
- assertTrue(dn.getXferServer().getNumPeersXceiver() == dn.getXferServer()
|
|
|
+ assertTrue(dn0.getXferServer().getNumPeersXceiver() == dn0.getXferServer()
|
|
|
.getNumPeersXceiver());
|
|
|
s1.close();
|
|
|
s2.close();
|
|
|
s3.close();
|
|
|
- assertTrue(dn.getXferServer().getNumPeersXceiver() == dn.getXferServer()
|
|
|
+ assertTrue(dn0.getXferServer().getNumPeersXceiver() == dn0.getXferServer()
|
|
|
.getNumPeersXceiver());
|
|
|
client1.close();
|
|
|
client2.close();
|
|
@@ -291,4 +303,143 @@ public class TestDataNodeRollingUpgrade {
|
|
|
shutdownCluster();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Support for layout version change with rolling upgrade was
|
|
|
+ * added by HDFS-6800 and HDFS-6981.
|
|
|
+ */
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testWithLayoutChangeAndFinalize() throws Exception {
|
|
|
+ final long seed = 0x600DF00D;
|
|
|
+ try {
|
|
|
+ startCluster();
|
|
|
+
|
|
|
+ Path[] paths = new Path[3];
|
|
|
+ File[] blockFiles = new File[3];
|
|
|
+
|
|
|
+ // Create two files in DFS.
|
|
|
+ for (int i = 0; i < 2; ++i) {
|
|
|
+ paths[i] = new Path("/" + GenericTestUtils.getMethodName() + "." + i + ".dat");
|
|
|
+ DFSTestUtil.createFile(fs, paths[i], BLOCK_SIZE, (short) 2, seed);
|
|
|
+ }
|
|
|
+
|
|
|
+ startRollingUpgrade();
|
|
|
+
|
|
|
+ // Delete the first file. The DN will save its block files in trash.
|
|
|
+ blockFiles[0] = getBlockForFile(paths[0], true);
|
|
|
+ File trashFile0 = getTrashFileForBlock(blockFiles[0], false);
|
|
|
+ deleteAndEnsureInTrash(paths[0], blockFiles[0], trashFile0);
|
|
|
+
|
|
|
+ // Restart the DN with a new layout version to trigger layout upgrade.
|
|
|
+ LOG.info("Shutting down the Datanode");
|
|
|
+ MiniDFSCluster.DataNodeProperties dnprop = cluster.stopDataNode(0);
|
|
|
+ DFSTestUtil.addDataNodeLayoutVersion(
|
|
|
+ DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1,
|
|
|
+ "Test Layout for TestDataNodeRollingUpgrade");
|
|
|
+ LOG.info("Restarting the DataNode");
|
|
|
+ cluster.restartDataNode(dnprop, true);
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ dn0 = cluster.getDataNodes().get(0);
|
|
|
+ LOG.info("The DN has been restarted");
|
|
|
+ assertFalse(trashFile0.exists());
|
|
|
+ assertFalse(dn0.getStorage().getBPStorage(blockPoolId).isTrashAllowed(blockFiles[0]));
|
|
|
+
|
|
|
+ // Ensure that the block file for the first file was moved from 'trash' to 'previous'.
|
|
|
+ assertTrue(isBlockFileInPrevious(blockFiles[0]));
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+
|
|
|
+ // Delete the second file. Ensure that its block file is in previous.
|
|
|
+ blockFiles[1] = getBlockForFile(paths[1], true);
|
|
|
+ fs.delete(paths[1], false);
|
|
|
+ assertTrue(isBlockFileInPrevious(blockFiles[1]));
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+
|
|
|
+ // Rollback and ensure that neither block file exists in trash or previous.
|
|
|
+ finalizeRollingUpgrade();
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+ assertFalse(isBlockFileInPrevious(blockFiles[0]));
|
|
|
+ assertFalse(isBlockFileInPrevious(blockFiles[1]));
|
|
|
+ } finally {
|
|
|
+ shutdownCluster();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Support for layout version change with rolling upgrade was
|
|
|
+ * added by HDFS-6800 and HDFS-6981.
|
|
|
+ */
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testWithLayoutChangeAndRollback() throws Exception {
|
|
|
+ final long seed = 0x600DF00D;
|
|
|
+ try {
|
|
|
+ startCluster();
|
|
|
+
|
|
|
+ Path[] paths = new Path[3];
|
|
|
+ File[] blockFiles = new File[3];
|
|
|
+
|
|
|
+ // Create two files in DFS.
|
|
|
+ for (int i = 0; i < 2; ++i) {
|
|
|
+ paths[i] = new Path("/" + GenericTestUtils.getMethodName() + "." + i + ".dat");
|
|
|
+ DFSTestUtil.createFile(fs, paths[i], BLOCK_SIZE, (short) 1, seed);
|
|
|
+ }
|
|
|
+
|
|
|
+ startRollingUpgrade();
|
|
|
+
|
|
|
+ // Delete the first file. The DN will save its block files in trash.
|
|
|
+ blockFiles[0] = getBlockForFile(paths[0], true);
|
|
|
+ File trashFile0 = getTrashFileForBlock(blockFiles[0], false);
|
|
|
+ deleteAndEnsureInTrash(paths[0], blockFiles[0], trashFile0);
|
|
|
+
|
|
|
+ // Restart the DN with a new layout version to trigger layout upgrade.
|
|
|
+ LOG.info("Shutting down the Datanode");
|
|
|
+ MiniDFSCluster.DataNodeProperties dnprop = cluster.stopDataNode(0);
|
|
|
+ DFSTestUtil.addDataNodeLayoutVersion(
|
|
|
+ DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1,
|
|
|
+ "Test Layout for TestDataNodeRollingUpgrade");
|
|
|
+ LOG.info("Restarting the DataNode");
|
|
|
+ cluster.restartDataNode(dnprop, true);
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ dn0 = cluster.getDataNodes().get(0);
|
|
|
+ LOG.info("The DN has been restarted");
|
|
|
+ assertFalse(trashFile0.exists());
|
|
|
+ assertFalse(dn0.getStorage().getBPStorage(blockPoolId).isTrashAllowed(blockFiles[0]));
|
|
|
+
|
|
|
+ // Ensure that the block file for the first file was moved from 'trash' to 'previous'.
|
|
|
+ assertTrue(isBlockFileInPrevious(blockFiles[0]));
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+
|
|
|
+ // Delete the second file. Ensure that its block file is in previous.
|
|
|
+ blockFiles[1] = getBlockForFile(paths[1], true);
|
|
|
+ fs.delete(paths[1], false);
|
|
|
+ assertTrue(isBlockFileInPrevious(blockFiles[1]));
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+
|
|
|
+ // Create and delete a third file. Its block file should not be
|
|
|
+ // in either trash or previous after deletion.
|
|
|
+ paths[2] = new Path("/" + GenericTestUtils.getMethodName() + ".2.dat");
|
|
|
+ DFSTestUtil.createFile(fs, paths[2], BLOCK_SIZE, (short) 1, seed);
|
|
|
+ blockFiles[2] = getBlockForFile(paths[2], true);
|
|
|
+ fs.delete(paths[2], false);
|
|
|
+ assertFalse(isBlockFileInPrevious(blockFiles[2]));
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+
|
|
|
+ // Rollback and ensure that the first two file contents were restored.
|
|
|
+ rollbackRollingUpgrade();
|
|
|
+ for (int i = 0; i < 2; ++i) {
|
|
|
+ byte[] actual = DFSTestUtil.readFileBuffer(fs, paths[i]);
|
|
|
+ byte[] calculated = DFSTestUtil.calculateFileContentsFromSeed(seed, BLOCK_SIZE);
|
|
|
+ assertArrayEquals(actual, calculated);
|
|
|
+ }
|
|
|
+
|
|
|
+ // And none of the block files must be in previous or trash.
|
|
|
+ assertFalse(isTrashRootPresent());
|
|
|
+ for (int i = 0; i < 3; ++i) {
|
|
|
+ assertFalse(isBlockFileInPrevious(blockFiles[i]));
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ shutdownCluster();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|