|
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|
@@ -69,20 +68,16 @@ import org.mockito.Mockito;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
|
|
|
/**
|
|
/**
|
|
- * This test simulates a variety of situations when blocks are being
|
|
|
|
- * intentionally corrupted, unexpectedly modified, and so on before a block
|
|
|
|
- * report is happening.
|
|
|
|
|
|
+ * This is the base class for simulating a variety of situations
|
|
|
|
+ * when blocks are being intentionally corrupted, unexpectedly modified,
|
|
|
|
+ * and so on before a block report is happening.
|
|
*
|
|
*
|
|
- * For each test case it runs two variations:
|
|
|
|
- * #1 - For a given DN, the first variation sends block reports for all
|
|
|
|
- * storages in a single call to the NN.
|
|
|
|
- * #2 - For a given DN, the second variation sends block reports for each
|
|
|
|
- * storage in a separate call.
|
|
|
|
- *
|
|
|
|
- * The behavior should be the same in either variation.
|
|
|
|
|
|
+ * By overriding {@link #sendBlockReports}, derived classes can test
|
|
|
|
+ * different variations of how block reports are split across storages
|
|
|
|
+ * and messages.
|
|
*/
|
|
*/
|
|
-public class TestBlockReport {
|
|
|
|
- public static final Log LOG = LogFactory.getLog(TestBlockReport.class);
|
|
|
|
|
|
+public abstract class BlockReportTestBase {
|
|
|
|
+ public static final Log LOG = LogFactory.getLog(BlockReportTestBase.class);
|
|
|
|
|
|
private static short REPL_FACTOR = 1;
|
|
private static short REPL_FACTOR = 1;
|
|
private static final int RAND_LIMIT = 2000;
|
|
private static final int RAND_LIMIT = 2000;
|
|
@@ -91,12 +86,11 @@ public class TestBlockReport {
|
|
private static final int DN_N0 = 0;
|
|
private static final int DN_N0 = 0;
|
|
private static final int FILE_START = 0;
|
|
private static final int FILE_START = 0;
|
|
|
|
|
|
- static final int BLOCK_SIZE = 1024;
|
|
|
|
- static final int NUM_BLOCKS = 10;
|
|
|
|
- static final int FILE_SIZE = NUM_BLOCKS * BLOCK_SIZE + 1;
|
|
|
|
- static String bpid;
|
|
|
|
|
|
+ private static final int BLOCK_SIZE = 1024;
|
|
|
|
+ private static final int NUM_BLOCKS = 10;
|
|
|
|
+ private static final int FILE_SIZE = NUM_BLOCKS * BLOCK_SIZE + 1;
|
|
|
|
|
|
- private MiniDFSCluster cluster;
|
|
|
|
|
|
+ protected MiniDFSCluster cluster;
|
|
private DistributedFileSystem fs;
|
|
private DistributedFileSystem fs;
|
|
|
|
|
|
private static Random rand = new Random(RAND_LIMIT);
|
|
private static Random rand = new Random(RAND_LIMIT);
|
|
@@ -112,8 +106,7 @@ public class TestBlockReport {
|
|
public void startUpCluster() throws IOException {
|
|
public void startUpCluster() throws IOException {
|
|
REPL_FACTOR = 1; //Reset if case a test has modified the value
|
|
REPL_FACTOR = 1; //Reset if case a test has modified the value
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
|
|
- fs = (DistributedFileSystem) cluster.getFileSystem();
|
|
|
|
- bpid = cluster.getNamesystem().getBlockPoolId();
|
|
|
|
|
|
+ fs = cluster.getFileSystem();
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
@@ -123,6 +116,15 @@ public class TestBlockReport {
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ protected static void resetConfiguration() {
|
|
|
|
+ conf = new Configuration();
|
|
|
|
+ int customPerChecksumSize = 512;
|
|
|
|
+ int customBlockSize = customPerChecksumSize * 3;
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, DN_RESCAN_INTERVAL);
|
|
|
|
+ }
|
|
|
|
+
|
|
// Generate a block report, optionally corrupting the generation
|
|
// Generate a block report, optionally corrupting the generation
|
|
// stamp and/or length of one block.
|
|
// stamp and/or length of one block.
|
|
private static StorageBlockReport[] getBlockReports(
|
|
private static StorageBlockReport[] getBlockReports(
|
|
@@ -172,106 +174,11 @@ public class TestBlockReport {
|
|
* @param dnR
|
|
* @param dnR
|
|
* @param poolId
|
|
* @param poolId
|
|
* @param reports
|
|
* @param reports
|
|
- * @param needtoSplit
|
|
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- private void sendBlockReports(DatanodeRegistration dnR, String poolId,
|
|
|
|
- StorageBlockReport[] reports, boolean needtoSplit) throws IOException {
|
|
|
|
- if (!needtoSplit) {
|
|
|
|
- LOG.info("Sending combined block reports for " + dnR);
|
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
|
- } else {
|
|
|
|
- for (StorageBlockReport report : reports) {
|
|
|
|
- LOG.info("Sending block report for storage " + report.getStorage().getStorageID());
|
|
|
|
- StorageBlockReport[] singletonReport = { report };
|
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, singletonReport);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ protected abstract void sendBlockReports(DatanodeRegistration dnR, String poolId,
|
|
|
|
+ StorageBlockReport[] reports) throws IOException;
|
|
|
|
|
|
- /**
|
|
|
|
- * Test variations blockReport_01 through blockReport_09 with combined
|
|
|
|
- * and split block reports.
|
|
|
|
- */
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_01() throws IOException {
|
|
|
|
- blockReport_01(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_01() throws IOException {
|
|
|
|
- blockReport_01(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_02() throws IOException {
|
|
|
|
- blockReport_02(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_02() throws IOException {
|
|
|
|
- blockReport_02(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_03() throws IOException {
|
|
|
|
- blockReport_03(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_03() throws IOException {
|
|
|
|
- blockReport_03(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_04() throws IOException {
|
|
|
|
- blockReport_04(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_04() throws IOException {
|
|
|
|
- blockReport_04(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_06() throws Exception {
|
|
|
|
- blockReport_06(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_06() throws Exception {
|
|
|
|
- blockReport_06(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_07() throws Exception {
|
|
|
|
- blockReport_07(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_07() throws Exception {
|
|
|
|
- blockReport_07(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_08() throws Exception {
|
|
|
|
- blockReport_08(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_08() throws Exception {
|
|
|
|
- blockReport_08(true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportCombined_09() throws Exception {
|
|
|
|
- blockReport_09(false);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void blockReportSplit_09() throws Exception {
|
|
|
|
- blockReport_09(true);
|
|
|
|
- }
|
|
|
|
/**
|
|
/**
|
|
* Test write a file, verifies and closes it. Then the length of the blocks
|
|
* Test write a file, verifies and closes it. Then the length of the blocks
|
|
* are messed up and BlockReport is forced.
|
|
* are messed up and BlockReport is forced.
|
|
@@ -279,7 +186,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws java.io.IOException on an error
|
|
* @throws java.io.IOException on an error
|
|
*/
|
|
*/
|
|
- private void blockReport_01(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_01() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
|
@@ -312,7 +220,7 @@ public class TestBlockReport {
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
|
|
|
|
List<LocatedBlock> blocksAfterReport =
|
|
List<LocatedBlock> blocksAfterReport =
|
|
DFSTestUtil.getAllBlocks(fs.open(filePath));
|
|
DFSTestUtil.getAllBlocks(fs.open(filePath));
|
|
@@ -338,7 +246,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of errors
|
|
* @throws IOException in case of errors
|
|
*/
|
|
*/
|
|
- private void blockReport_02(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_02() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
LOG.info("Running test " + METHOD_NAME);
|
|
LOG.info("Running test " + METHOD_NAME);
|
|
|
|
|
|
@@ -393,7 +302,7 @@ public class TestBlockReport {
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
|
|
|
|
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
|
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
|
.getBlockManager());
|
|
.getBlockManager());
|
|
@@ -414,17 +323,18 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- private void blockReport_03(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_03() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
-
|
|
|
|
|
|
+
|
|
// all blocks belong to the same file, hence same BP
|
|
// all blocks belong to the same file, hence same BP
|
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
|
|
|
|
assertThat("Wrong number of corrupt blocks",
|
|
assertThat("Wrong number of corrupt blocks",
|
|
@@ -441,7 +351,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- private void blockReport_04(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_04() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
DFSTestUtil.createFile(fs, filePath,
|
|
DFSTestUtil.createFile(fs, filePath,
|
|
@@ -459,7 +370,7 @@ public class TestBlockReport {
|
|
|
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
|
|
|
|
assertThat("Wrong number of corrupt blocks",
|
|
assertThat("Wrong number of corrupt blocks",
|
|
@@ -476,7 +387,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- private void blockReport_06(boolean splitBlockReports) throws Exception {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_06() throws Exception {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
final int DN_N1 = DN_N0 + 1;
|
|
final int DN_N1 = DN_N0 + 1;
|
|
@@ -489,7 +401,7 @@ public class TestBlockReport {
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
assertEquals("Wrong number of PendingReplication Blocks",
|
|
assertEquals("Wrong number of PendingReplication Blocks",
|
|
0, cluster.getNamesystem().getUnderReplicatedBlocks());
|
|
0, cluster.getNamesystem().getUnderReplicatedBlocks());
|
|
@@ -508,7 +420,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- private void blockReport_07(boolean splitBlockReports) throws Exception {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_07() throws Exception {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
final int DN_N1 = DN_N0 + 1;
|
|
final int DN_N1 = DN_N0 + 1;
|
|
@@ -522,7 +435,7 @@ public class TestBlockReport {
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
|
|
|
|
assertThat("Wrong number of corrupt blocks",
|
|
assertThat("Wrong number of corrupt blocks",
|
|
@@ -533,7 +446,7 @@ public class TestBlockReport {
|
|
cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
|
|
cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
|
|
|
|
|
|
reports = getBlockReports(dn, poolId, true, true);
|
|
reports = getBlockReports(dn, poolId, true, true);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
|
|
|
|
assertThat("Wrong number of corrupt blocks",
|
|
assertThat("Wrong number of corrupt blocks",
|
|
@@ -559,7 +472,8 @@ public class TestBlockReport {
|
|
*
|
|
*
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- private void blockReport_08(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_08() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
final int DN_N1 = DN_N0 + 1;
|
|
final int DN_N1 = DN_N0 + 1;
|
|
@@ -578,13 +492,13 @@ public class TestBlockReport {
|
|
bc.start();
|
|
bc.start();
|
|
|
|
|
|
waitForTempReplica(bl, DN_N1);
|
|
waitForTempReplica(bl, DN_N1);
|
|
-
|
|
|
|
|
|
+
|
|
// all blocks belong to the same file, hence same BP
|
|
// all blocks belong to the same file, hence same BP
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
blocks.size(), cluster.getNamesystem().getPendingReplicationBlocks());
|
|
blocks.size(), cluster.getNamesystem().getPendingReplicationBlocks());
|
|
@@ -600,7 +514,8 @@ public class TestBlockReport {
|
|
// Similar to BlockReport_08 but corrupts GS and len of the TEMPORARY's
|
|
// Similar to BlockReport_08 but corrupts GS and len of the TEMPORARY's
|
|
// replica block. Expect the same behaviour: NN should simply ignore this
|
|
// replica block. Expect the same behaviour: NN should simply ignore this
|
|
// block
|
|
// block
|
|
- private void blockReport_09(boolean splitBlockReports) throws IOException {
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
|
|
+ public void blockReport_09() throws IOException {
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
final int DN_N1 = DN_N0 + 1;
|
|
final int DN_N1 = DN_N0 + 1;
|
|
@@ -620,17 +535,17 @@ public class TestBlockReport {
|
|
bc.start();
|
|
bc.start();
|
|
|
|
|
|
waitForTempReplica(bl, DN_N1);
|
|
waitForTempReplica(bl, DN_N1);
|
|
-
|
|
|
|
|
|
+
|
|
// all blocks belong to the same file, hence same BP
|
|
// all blocks belong to the same file, hence same BP
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, true);
|
|
StorageBlockReport[] reports = getBlockReports(dn, poolId, true, true);
|
|
- sendBlockReports(dnR, poolId, reports, splitBlockReports);
|
|
|
|
|
|
+ sendBlockReports(dnR, poolId, reports);
|
|
printStats();
|
|
printStats();
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
2, cluster.getNamesystem().getPendingReplicationBlocks());
|
|
2, cluster.getNamesystem().getPendingReplicationBlocks());
|
|
-
|
|
|
|
|
|
+
|
|
try {
|
|
try {
|
|
bc.join();
|
|
bc.join();
|
|
} catch (InterruptedException e) {}
|
|
} catch (InterruptedException e) {}
|
|
@@ -638,7 +553,7 @@ public class TestBlockReport {
|
|
resetConfiguration(); // return the initial state of the configuration
|
|
resetConfiguration(); // return the initial state of the configuration
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Test for the case where one of the DNs in the pipeline is in the
|
|
* Test for the case where one of the DNs in the pipeline is in the
|
|
* process of doing a block report exactly when the block is closed.
|
|
* process of doing a block report exactly when the block is closed.
|
|
@@ -648,7 +563,7 @@ public class TestBlockReport {
|
|
* corrupt.
|
|
* corrupt.
|
|
* This is a regression test for HDFS-2791.
|
|
* This is a regression test for HDFS-2791.
|
|
*/
|
|
*/
|
|
- @Test
|
|
|
|
|
|
+ @Test(timeout=300000)
|
|
public void testOneReplicaRbwReportArrivesAfterBlockCompleted() throws Exception {
|
|
public void testOneReplicaRbwReportArrivesAfterBlockCompleted() throws Exception {
|
|
final CountDownLatch brFinished = new CountDownLatch(1);
|
|
final CountDownLatch brFinished = new CountDownLatch(1);
|
|
DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG) {
|
|
DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG) {
|
|
@@ -663,7 +578,7 @@ public class TestBlockReport {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
};
|
|
};
|
|
-
|
|
|
|
|
|
+
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
|
@@ -671,9 +586,9 @@ public class TestBlockReport {
|
|
// what happens when one of the DNs is slowed for some reason.
|
|
// what happens when one of the DNs is slowed for some reason.
|
|
REPL_FACTOR = 2;
|
|
REPL_FACTOR = 2;
|
|
startDNandWait(null, false);
|
|
startDNandWait(null, false);
|
|
-
|
|
|
|
|
|
+
|
|
NameNode nn = cluster.getNameNode();
|
|
NameNode nn = cluster.getNameNode();
|
|
-
|
|
|
|
|
|
+
|
|
FSDataOutputStream out = fs.create(filePath, REPL_FACTOR);
|
|
FSDataOutputStream out = fs.create(filePath, REPL_FACTOR);
|
|
try {
|
|
try {
|
|
AppendTestUtil.write(out, 0, 10);
|
|
AppendTestUtil.write(out, 0, 10);
|
|
@@ -684,19 +599,19 @@ public class TestBlockReport {
|
|
DataNode dn = cluster.getDataNodes().get(0);
|
|
DataNode dn = cluster.getDataNodes().get(0);
|
|
DatanodeProtocolClientSideTranslatorPB spy =
|
|
DatanodeProtocolClientSideTranslatorPB spy =
|
|
DataNodeTestUtils.spyOnBposToNN(dn, nn);
|
|
DataNodeTestUtils.spyOnBposToNN(dn, nn);
|
|
-
|
|
|
|
|
|
+
|
|
Mockito.doAnswer(delayer)
|
|
Mockito.doAnswer(delayer)
|
|
.when(spy).blockReport(
|
|
.when(spy).blockReport(
|
|
Mockito.<DatanodeRegistration>anyObject(),
|
|
Mockito.<DatanodeRegistration>anyObject(),
|
|
Mockito.anyString(),
|
|
Mockito.anyString(),
|
|
Mockito.<StorageBlockReport[]>anyObject());
|
|
Mockito.<StorageBlockReport[]>anyObject());
|
|
-
|
|
|
|
|
|
+
|
|
// Force a block report to be generated. The block report will have
|
|
// Force a block report to be generated. The block report will have
|
|
// an RBW replica in it. Wait for the RPC to be sent, but block
|
|
// an RBW replica in it. Wait for the RPC to be sent, but block
|
|
// it before it gets to the NN.
|
|
// it before it gets to the NN.
|
|
dn.scheduleAllBlockReport(0);
|
|
dn.scheduleAllBlockReport(0);
|
|
delayer.waitForCall();
|
|
delayer.waitForCall();
|
|
-
|
|
|
|
|
|
+
|
|
} finally {
|
|
} finally {
|
|
IOUtils.closeStream(out);
|
|
IOUtils.closeStream(out);
|
|
}
|
|
}
|
|
@@ -705,22 +620,22 @@ public class TestBlockReport {
|
|
// state.
|
|
// state.
|
|
delayer.proceed();
|
|
delayer.proceed();
|
|
brFinished.await();
|
|
brFinished.await();
|
|
-
|
|
|
|
|
|
+
|
|
// Verify that no replicas are marked corrupt, and that the
|
|
// Verify that no replicas are marked corrupt, and that the
|
|
// file is still readable.
|
|
// file is still readable.
|
|
BlockManagerTestUtil.updateState(nn.getNamesystem().getBlockManager());
|
|
BlockManagerTestUtil.updateState(nn.getNamesystem().getBlockManager());
|
|
assertEquals(0, nn.getNamesystem().getCorruptReplicaBlocks());
|
|
assertEquals(0, nn.getNamesystem().getCorruptReplicaBlocks());
|
|
DFSTestUtil.readFile(fs, filePath);
|
|
DFSTestUtil.readFile(fs, filePath);
|
|
-
|
|
|
|
|
|
+
|
|
// Ensure that the file is readable even from the DN that we futzed with.
|
|
// Ensure that the file is readable even from the DN that we futzed with.
|
|
cluster.stopDataNode(1);
|
|
cluster.stopDataNode(1);
|
|
- DFSTestUtil.readFile(fs, filePath);
|
|
|
|
|
|
+ DFSTestUtil.readFile(fs, filePath);
|
|
}
|
|
}
|
|
|
|
|
|
private void waitForTempReplica(Block bl, int DN_N1) throws IOException {
|
|
private void waitForTempReplica(Block bl, int DN_N1) throws IOException {
|
|
final boolean tooLongWait = false;
|
|
final boolean tooLongWait = false;
|
|
final int TIMEOUT = 40000;
|
|
final int TIMEOUT = 40000;
|
|
-
|
|
|
|
|
|
+
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
LOG.debug("Wait for datanode " + DN_N1 + " to appear");
|
|
LOG.debug("Wait for datanode " + DN_N1 + " to appear");
|
|
}
|
|
}
|
|
@@ -731,7 +646,7 @@ public class TestBlockReport {
|
|
LOG.debug("Total number of DNs " + cluster.getDataNodes().size());
|
|
LOG.debug("Total number of DNs " + cluster.getDataNodes().size());
|
|
}
|
|
}
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
-
|
|
|
|
|
|
+
|
|
// Look about specified DN for the replica of the block from 1st DN
|
|
// Look about specified DN for the replica of the block from 1st DN
|
|
final DataNode dn1 = cluster.getDataNodes().get(DN_N1);
|
|
final DataNode dn1 = cluster.getDataNodes().get(DN_N1);
|
|
String bpid = cluster.getNamesystem().getBlockPoolId();
|
|
String bpid = cluster.getNamesystem().getBlockPoolId();
|
|
@@ -789,7 +704,7 @@ public class TestBlockReport {
|
|
return blocks;
|
|
return blocks;
|
|
}
|
|
}
|
|
|
|
|
|
- private void startDNandWait(Path filePath, boolean waitReplicas)
|
|
|
|
|
|
+ private void startDNandWait(Path filePath, boolean waitReplicas)
|
|
throws IOException, InterruptedException, TimeoutException {
|
|
throws IOException, InterruptedException, TimeoutException {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Before next DN start: " + cluster.getDataNodes().size());
|
|
LOG.debug("Before next DN start: " + cluster.getDataNodes().size());
|
|
@@ -802,7 +717,7 @@ public class TestBlockReport {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
int lastDn = datanodes.size() - 1;
|
|
int lastDn = datanodes.size() - 1;
|
|
LOG.debug("New datanode "
|
|
LOG.debug("New datanode "
|
|
- + cluster.getDataNodes().get(lastDn).getDisplayName()
|
|
|
|
|
|
+ + cluster.getDataNodes().get(lastDn).getDisplayName()
|
|
+ " has been started");
|
|
+ " has been started");
|
|
}
|
|
}
|
|
if (waitReplicas) {
|
|
if (waitReplicas) {
|
|
@@ -898,7 +813,7 @@ public class TestBlockReport {
|
|
((Log4JLogger) NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
|
|
((Log4JLogger) NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
|
|
((Log4JLogger) LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
|
|
((Log4JLogger) LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
|
|
((Log4JLogger) DataNode.LOG).getLogger().setLevel(Level.ALL);
|
|
((Log4JLogger) DataNode.LOG).getLogger().setLevel(Level.ALL);
|
|
- ((Log4JLogger) TestBlockReport.LOG).getLogger().setLevel(Level.ALL);
|
|
|
|
|
|
+ ((Log4JLogger) BlockReportTestBase.LOG).getLogger().setLevel(Level.ALL);
|
|
}
|
|
}
|
|
|
|
|
|
private Block findBlock(Path path, long size) throws IOException {
|
|
private Block findBlock(Path path, long size) throws IOException {
|
|
@@ -918,11 +833,11 @@ public class TestBlockReport {
|
|
|
|
|
|
private class BlockChecker extends Thread {
|
|
private class BlockChecker extends Thread {
|
|
Path filePath;
|
|
Path filePath;
|
|
-
|
|
|
|
|
|
+
|
|
public BlockChecker(final Path filePath) {
|
|
public BlockChecker(final Path filePath) {
|
|
this.filePath = filePath;
|
|
this.filePath = filePath;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void run() {
|
|
public void run() {
|
|
try {
|
|
try {
|
|
@@ -933,13 +848,4 @@ public class TestBlockReport {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- private static void resetConfiguration() {
|
|
|
|
- conf = new Configuration();
|
|
|
|
- int customPerChecksumSize = 512;
|
|
|
|
- int customBlockSize = customPerChecksumSize * 3;
|
|
|
|
- conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
|
- conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
|
- conf.setLong(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, DN_RESCAN_INTERVAL);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|