HDFS-15308. TestReconstructStripedFile#testNNSendsErasureCodingTasks fails intermittently. Contributed by Hemanth Boyina.

This commit is contained in:
Ayush Saxena 2020-12-22 02:07:51 +05:30
parent a35fc3871b
commit ba496f61f8

View File

@ -468,7 +468,7 @@ private void testNNSendsErasureCodingTasks(int deadDN) throws Exception {
final int numDataNodes = dnNum + 1;
conf.setInt(
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 10);
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 20);
conf.setInt(DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_THREADS_KEY,
2);
@ -504,8 +504,9 @@ private void testNNSendsErasureCodingTasks(int deadDN) throws Exception {
// Make sure that all pending reconstruction tasks can be processed.
while (ns.getPendingReconstructionBlocks() > 0) {
long timeoutPending = ns.getNumTimedOutPendingReconstructions();
assertTrue(String.format("Found %d timeout pending reconstruction tasks",
timeoutPending), timeoutPending == 0);
assertEquals(String
.format("Found %d timeout pending reconstruction tasks",
timeoutPending), 0, timeoutPending);
Thread.sleep(1000);
}