HDFS-15654. TestBPOfferService#testMissBlocksWhenReregister fails intermittently (#2419)

This commit is contained in:
Ahmed Hussein 2020-10-28 18:24:34 -05:00 committed by GitHub
parent bab5bf9743
commit 324879127a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

View File

@ -34,6 +34,7 @@
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertSame; import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@ -47,6 +48,7 @@
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -285,21 +287,18 @@ public void testMissBlocksWhenReregister() throws Exception {
int totalTestBlocks = 4000; int totalTestBlocks = 4000;
Thread addNewBlockThread = null; Thread addNewBlockThread = null;
final AtomicInteger count = new AtomicInteger(0); final AtomicInteger count = new AtomicInteger(0);
DataNodeFaultInjector prevDNFaultInjector = null;
try { try {
waitForBothActors(bpos); waitForBothActors(bpos);
waitForInitialization(bpos); waitForInitialization(bpos);
prevDNFaultInjector = DataNodeFaultInjector.get();
DataNodeFaultInjector.set(new DataNodeFaultInjector() { DataNodeFaultInjector.set(new DataNodeFaultInjector() {
public void blockUtilSendFullBlockReport() { public void blockUtilSendFullBlockReport() {
try { try {
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> count.get() > 2000,
if(count.get() > 2000) { 100, 1000);
return true;
}
return false;
}, 100, 1000);
} catch (Exception e) { } catch (Exception e) {
e.printStackTrace(); LOG.error("error DataNodeFaultInjector", e);
} }
} }
}); });
@ -318,45 +317,41 @@ public void blockUtilSendFullBlockReport() {
count.addAndGet(1); count.addAndGet(1);
Thread.sleep(1); Thread.sleep(1);
} catch (Exception e) { } catch (Exception e) {
e.printStackTrace(); LOG.error("error addNewBlockThread", e);
} }
} }
}); });
addNewBlockThread.start(); addNewBlockThread.start();
// Make sure that generate blocks for DataNode and IBR not empty now. // Make sure that generate blocks for DataNode and IBR not empty now.
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> count.get() > 0, 100, 1000);
if(count.get() > 0) {
return true;
}
return false;
}, 100, 1000);
// Trigger re-register using DataNode Command. // Trigger re-register using DataNode Command.
datanodeCommands[0] = new DatanodeCommand[]{RegisterCommand.REGISTER}; datanodeCommands[0] = new DatanodeCommand[]{RegisterCommand.REGISTER};
bpos.triggerHeartbeatForTests(); bpos.triggerHeartbeatForTests();
try {
GenericTestUtils.waitFor(() -> {
if(fullBlockReportCount == totalTestBlocks ||
incrBlockReportCount == totalTestBlocks) {
return true;
}
return false;
}, 1000, 15000);
} catch (Exception e) {}
// Verify FBR/IBR count is equal to generate number.
assertTrue(fullBlockReportCount == totalTestBlocks ||
incrBlockReportCount == totalTestBlocks);
} finally {
addNewBlockThread.join(); addNewBlockThread.join();
addNewBlockThread = null;
// Verify FBR/IBR count is equal to generate number.
try {
GenericTestUtils.waitFor(() ->
(fullBlockReportCount == totalTestBlocks ||
incrBlockReportCount == totalTestBlocks), 1000, 15000);
} catch (Exception e) {
fail(String.format("Timed out wait for IBR counts FBRCount = %d,"
+ " IBRCount = %d; expected = %d. Exception: %s",
fullBlockReportCount, incrBlockReportCount, totalTestBlocks,
e.getMessage()));
}
} finally {
if (addNewBlockThread != null) {
addNewBlockThread.interrupt();
}
bpos.stop(); bpos.stop();
bpos.join(); bpos.join();
DataNodeFaultInjector.set(new DataNodeFaultInjector() { DataNodeFaultInjector.set(prevDNFaultInjector);
public void blockUtilSendFullBlockReport() {}
});
} }
} }